From 25c0739bad7222d45b4818c7bf6987521a3509d2 Mon Sep 17 00:00:00 2001 From: folone Date: Mon, 14 Jan 2013 09:52:11 +0100 Subject: Moved to scala 2.10.0. Notable changes are: - akka 2.0.3 → 2.1.0 - spray 1.0-M1 → 1.1-M7 For now the repl subproject is commented out, as scala reflection api changed very much since the introduction of macros. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/scala/spark/CacheTracker.scala | 6 +-- core/src/main/scala/spark/MapOutputTracker.scala | 6 +-- .../scala/spark/api/java/function/Function.java | 8 ++-- .../scala/spark/api/java/function/Function2.java | 9 +++-- .../api/java/function/PairFlatMapFunction.java | 12 +++--- .../spark/api/java/function/PairFunction.java | 12 +++--- .../src/main/scala/spark/deploy/JsonProtocol.scala | 2 +- .../main/scala/spark/deploy/client/Client.scala | 4 +- .../main/scala/spark/deploy/master/Master.scala | 2 +- .../scala/spark/deploy/master/MasterWebUI.scala | 45 +++++++++++----------- .../main/scala/spark/deploy/worker/Worker.scala | 2 +- .../scala/spark/deploy/worker/WorkerWebUI.scala | 43 +++++++++++---------- .../scala/spark/network/ConnectionManager.scala | 6 +-- .../spark/network/ConnectionManagerTest.scala | 4 +- .../cluster/StandaloneSchedulerBackend.scala | 4 +- .../main/scala/spark/storage/BlockManager.scala | 6 +-- .../scala/spark/storage/BlockManagerMaster.scala | 5 +-- .../spark/storage/BlockManagerMasterActor.scala | 4 +- core/src/main/scala/spark/util/AkkaUtils.scala | 37 +++++++++--------- core/src/test/scala/spark/CacheTrackerSuite.scala | 11 +++--- 20 files changed, 115 insertions(+), 113 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/spark/CacheTracker.scala b/core/src/main/scala/spark/CacheTracker.scala index 04c26b2e40..a73438208a 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 70eb9f702e..08d2956782 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 dae8295f21..9f6ab9a592 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 @@ import java.io.Serializable; 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 69bf12c8c9..b32c178aa3 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 b3cc4df6aa..296a3b5044 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 9fc6df4b88..e3f94788e2 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 732fa08064..12069947fc 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 90fe9508cd..0aee759796 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 6ecebe626a..e034312c12 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 458ee2d665..a4dadb6ef9 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 7c9e588ea2..ec25a19e7b 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 f9489d99fc..7dd1781900 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 36c01ad629..04b303afe0 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 533e4610f3..1b5b0935c2 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 eeaae23dc8..03dc5f4b9b 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 7a8ac10cdd..7a1344668f 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 a3d8671834..46e1860d09 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 f4d026da33..9bf9161c31 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 e67cb0336d..6fd9aa70fc 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 467605981b..d0c2bd47fc 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) -- cgit v1.2.3 From fd6e51deec83f01be3db41e84255329eedbe15da Mon Sep 17 00:00:00 2001 From: folone Date: Sun, 20 Jan 2013 17:02:58 +0100 Subject: Fixed the failing test. --- core/src/test/scala/spark/AccumulatorSuite.scala | 33 +++++++++++------------- 1 file changed, 15 insertions(+), 18 deletions(-) (limited to 'core/src') diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/spark/AccumulatorSuite.scala index d8be99dde7..9f5335978f 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 } } -- cgit v1.2.3 From ad88f083a627ba38e99b1b135a82a1fcfd107444 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 24 Apr 2013 18:08:26 +0530 Subject: scala 2.10 and master merge --- core/src/main/scala/spark/deploy/master/Master.scala | 3 ++- .../main/scala/spark/deploy/master/MasterWebUI.scala | 20 ++------------------ core/src/main/scala/spark/deploy/worker/Worker.scala | 6 ++++-- .../main/scala/spark/deploy/worker/WorkerWebUI.scala | 11 +++++++---- core/src/main/scala/spark/storage/BlockManager.scala | 2 +- .../main/scala/spark/storage/BlockManagerUI.scala | 15 ++++++++------- core/src/main/scala/spark/util/AkkaUtils.scala | 1 - core/src/test/scala/spark/AccumulatorSuite.scala | 2 +- core/src/test/scala/spark/DistributedSuite.scala | 8 ++++---- project/SparkBuild.scala | 8 ++++---- repl/src/main/scala/spark/repl/SparkILoop.scala | 15 --------------- repl/src/main/scala/spark/repl/SparkILoopInit.scala | 2 +- repl/src/test/scala/spark/repl/ReplSuiteMixin.scala | 1 + run | 2 +- .../scala/spark/streaming/NetworkInputTracker.scala | 4 ++-- .../streaming/dstream/NetworkInputDStream.scala | 6 ++++-- .../spark/streaming/receivers/ActorReceiver.scala | 19 +++++++++---------- .../spark/streaming/receivers/ZeroMQReceiver.scala | 2 +- 18 files changed, 52 insertions(+), 75 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index e070a15a54..d1428bcfc6 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -3,7 +3,7 @@ package spark.deploy.master import akka.actor._ import akka.actor.Terminated import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} -import akka.util.duration._ +import scala.concurrent.duration._ import java.text.SimpleDateFormat import java.util.Date @@ -50,6 +50,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) startWebUi() + import context.dispatcher context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) } diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 59d59dde78..fe859d48c3 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -27,6 +27,7 @@ class MasterWebUI(master: ActorRef)(implicit val context: ActorContext) extends implicit val timeout = Timeout(10 seconds) + val handler = { get { (path("") & parameters('format ?)) { @@ -52,27 +53,11 @@ class MasterWebUI(master: ActorRef)(implicit val context: ActorContext) extends masterState.completedApps.find(_.id == appId).getOrElse(null) }) } -<<<<<<< HEAD respondWithMediaType(`application/json`) { ctx => - ctx.complete(jobInfo.mapTo[JobInfo]) - } - case (jobId, _) => - 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 { - case Some(job) => spark.deploy.master.html.job_details.render(job) - case _ => null - } - } -======= - respondWithMediaType(MediaTypes.`application/json`) { ctx => ctx.complete(appInfo.mapTo[ApplicationInfo]) } case (appId, _) => - completeWith { + complete { val future = master ? RequestMasterState future.map { state => val masterState = state.asInstanceOf[MasterState] @@ -80,7 +65,6 @@ class MasterWebUI(master: ActorRef)(implicit val context: ActorContext) extends masterState.completedApps.find(_.id == appId).getOrElse(null) }) spark.deploy.master.html.app_details.render(app) ->>>>>>> 17e076de800ea0d4c55f2bd657348641f6f9c55b } } } diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 700d87b1c1..5bcf00443c 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -2,7 +2,7 @@ package spark.deploy.worker import scala.collection.mutable.{ArrayBuffer, HashMap} import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} -import akka.util.duration._ +import scala.concurrent.duration._ import spark.{Logging, Utils} import spark.util.AkkaUtils import spark.deploy._ @@ -15,6 +15,7 @@ import spark.deploy.RegisterWorkerFailed import spark.deploy.master.Master import java.io.File + private[spark] class Worker( ip: String, port: Int, @@ -81,7 +82,7 @@ private[spark] class Worker( } def startWebUi() { - val webUi = new WorkerWebUI(context.system, self, workDir) + val webUi = new WorkerWebUI(self, workDir) try { AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) } catch { @@ -95,6 +96,7 @@ private[spark] class Worker( case RegisteredWorker(url) => masterWebUiUrl = url logInfo("Successfully registered with master") + import context.dispatcher context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis) { master ! Heartbeat(workerId) } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index 99c3b506fa..33a2a9516e 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -18,8 +18,11 @@ import java.io.File * 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" +class WorkerWebUI(worker: ActorRef, workDir: File)(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(10 seconds) @@ -42,9 +45,9 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef, workDir: File) } } ~ path("log") { - parameters("jobId", "executorId", "logType") { (jobId, executorId, logType) => + parameters("appId", "executorId", "logType") { (appId, executorId, logType) => respondWithMediaType(`text/plain`) { - getFromFileName(workDir.getPath() + "/" + appId + "/" + executorId + "/" + logType) + getFromFile(workDir.getPath() + "/" + appId + "/" + executorId + "/" + logType) } } } ~ diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 8f737c5c6a..d3f6cd78dc 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -873,7 +873,7 @@ object BlockManager extends Logging { } def getHeartBeatFrequencyFromSystemProperties: Long = - System.getProperty("spark.storage.blockManagerHeartBeatMs", "5000").toLong + System.getProperty("spark.storage.blockManagerHeartBeatMs", "10000").toLong def getDisableHeartBeatsForTesting: Boolean = System.getProperty("spark.test.disableBlockManagerHeartBeat", "false").toBoolean diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 9e6721ec17..a3397a0fb4 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -2,9 +2,9 @@ package spark.storage import akka.actor.{ActorRef, ActorSystem} import akka.util.Timeout -import akka.util.duration._ -import cc.spray.typeconversion.TwirlSupport._ -import cc.spray.Directives +import scala.concurrent.duration._ +import spray.httpx.TwirlSupport._ +import spray.routing.Directives import spark.{Logging, SparkContext} import spark.util.AkkaUtils import spark.Utils @@ -17,7 +17,8 @@ private[spark] class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, sc: SparkContext) extends Directives with Logging { - val STATIC_RESOURCE_DIR = "spark/deploy/static" + implicit val implicitActorSystem = actorSystem + val STATIC_RESOURCE_DIR = "spark/deploy/static" implicit val timeout = Timeout(10 seconds) @@ -31,7 +32,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, // random port it bound to, so we have to try to find a local one by creating a socket. Utils.findFreePort() } - AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", port, handler, "BlockManagerHTTPServer") + AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", port, handler) logInfo("Started BlockManager web UI at http://%s:%d".format(Utils.localHostName(), port)) } catch { case e: Exception => @@ -43,7 +44,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, val handler = { get { path("") { - completeWith { + complete { // Request the current storage status from the Master val storageStatusList = sc.getExecutorStorageStatus // Calculate macro-level statistics @@ -58,7 +59,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, } ~ path("rdd") { parameter("id") { id => - completeWith { + complete { val prefix = "rdd_" + id.toString val storageStatusList = sc.getExecutorStorageStatus val filteredStorageStatusList = StorageUtils. diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 6f551b2b9c..70338ec4dc 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -32,7 +32,6 @@ private[spark] object AkkaUtils { val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt val lifecycleEvents = System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean - val lifecycleEvents = System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean val akkaConf = ConfigFactory.parseString(""" akka.daemonic = on akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/spark/AccumulatorSuite.scala index f59334a033..fb54ccb51e 100644 --- a/core/src/test/scala/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/spark/AccumulatorSuite.scala @@ -8,7 +8,7 @@ import scala.math.exp import scala.math.signum import spark.SparkContext._ -class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter with LocalSparkContext { +class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkContext { implicit def setAccum[A] = new AccumulableParam[mutable.Set[A], A] { diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 4104b33c8b..46b74fe5ee 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -257,7 +257,7 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter object DistributedSuite { // Indicates whether this JVM is marked for failure. var mark = false - + // Set by test to remember if we are in the driver program so we can assert // that we are not. var amMaster = false @@ -274,9 +274,9 @@ object DistributedSuite { // Act like an identity function, but if mark was set to true previously, fail, // crashing the entire JVM. def failOnMarkedIdentity(item: Boolean): Boolean = { - if (mark) { + if (mark) { System.exit(42) - } + } item - } + } } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 2571e54b04..7b61e2ba3e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -178,10 +178,10 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", - "com.github.sgroschupf" % "zkclient" % "0.1", - "org.twitter4j" % "twitter4j-stream" % "3.0.3", - "com.typesafe.akka" % "akka-zeromq" % "2.0.3" + "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", + "com.github.sgroschupf" % "zkclient" % "0.1", + "org.twitter4j" % "twitter4j-stream" % "3.0.3", + "com.typesafe.akka" % "akka-zeromq" % "2.1-M1" excludeAll(ExclusionRule(name = "akka-actor"), ExclusionRule(organization = "org.scala-lang")) ) ) ++ assemblySettings ++ extraAssemblySettings diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala index e83e403760..2b6e7b68bf 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/spark/repl/SparkILoop.scala @@ -222,21 +222,6 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } - /** Print a welcome message */ - def printWelcome() { - echo("""Welcome to - ____ __ - / __/__ ___ _____/ /__ - _\ \/ _ \/ _ `/ __/ '_/ - /___/ .__/\_,_/_/ /_/\_\ version 0.8.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]" diff --git a/repl/src/main/scala/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/spark/repl/SparkILoopInit.scala index 6ae535c4e6..8b7da3d3c6 100644 --- a/repl/src/main/scala/spark/repl/SparkILoopInit.scala +++ b/repl/src/main/scala/spark/repl/SparkILoopInit.scala @@ -24,7 +24,7 @@ trait SparkILoopInit { ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ - /___/ .__/\_,_/_/ /_/\_\ version 0.7.1-SNAPSHOT + /___/ .__/\_,_/_/ /_/\_\ version 0.8.0 /_/ """) import Properties._ diff --git a/repl/src/test/scala/spark/repl/ReplSuiteMixin.scala b/repl/src/test/scala/spark/repl/ReplSuiteMixin.scala index fd1a1b1e7c..8f439f0681 100644 --- a/repl/src/test/scala/spark/repl/ReplSuiteMixin.scala +++ b/repl/src/test/scala/spark/repl/ReplSuiteMixin.scala @@ -17,6 +17,7 @@ trait ReplSuiteMixin { val localIp = "127.0.1.2" val port = "7089" val sparkUrl = s"spark://$localIp:$port" + def setupStandaloneCluster() { future { Master.main(Array("-i", localIp, "-p", port, "--webui-port", "0")) } Thread.sleep(2000) diff --git a/run b/run index 4755d562a7..96c7f8a095 100755 --- a/run +++ b/run @@ -164,4 +164,4 @@ else EXTRA_ARGS="$JAVA_OPTS" fi -exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@" +exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@" diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala index b159d26c02..e5bb654578 100644 --- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala +++ b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala @@ -11,8 +11,8 @@ import scala.collection.mutable.Queue import akka.actor._ import akka.pattern.ask -import akka.util.duration._ -import akka.dispatch._ +import scala.concurrent.duration._ +// import akka.dispatch._ private[streaming] sealed trait NetworkInputTrackerMessage private[streaming] case class RegisterReceiver(streamId: Int, receiverActor: ActorRef) extends NetworkInputTrackerMessage diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index 7385474963..5347374730 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -7,13 +7,15 @@ import spark.rdd.BlockRDD import spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ import java.nio.ByteBuffer import akka.actor.{Props, Actor} import akka.pattern.ask -import akka.dispatch.Await -import akka.util.duration._ +import scala.concurrent.Await +import akka.util.Timeout + import spark.streaming.util.{RecurringTimer, SystemClock} import java.util.concurrent.ArrayBlockingQueue diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala index b3201d0b28..6c9e373de3 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala @@ -3,6 +3,8 @@ package spark.streaming.receivers import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy } import akka.actor.{ actorRef2Scala, ActorRef } import akka.actor.{ PossiblyHarmful, OneForOneStrategy } +import akka.actor.SupervisorStrategy._ +import scala.concurrent.duration._ import spark.storage.StorageLevel import spark.streaming.dstream.NetworkReceiver @@ -12,9 +14,6 @@ import java.util.concurrent.atomic.AtomicInteger /** A helper with set of defaults for supervisor strategy **/ object ReceiverSupervisorStrategy { - import akka.util.duration._ - import akka.actor.SupervisorStrategy._ - val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = 15 millis) { case _: RuntimeException ⇒ Restart @@ -27,10 +26,10 @@ object ReceiverSupervisorStrategy { * pushBlock API. * * @example {{{ - * class MyActor extends Actor with Receiver{ - * def receive { - * case anything :String ⇒ pushBlock(anything) - * } + * class MyActor extends Actor with Receiver{ + * def receive { + * case anything :String ⇒ pushBlock(anything) + * } * } * //Can be plugged in actorStream as follows * ssc.actorStream[String](Props(new MyActor),"MyActorReceiver") @@ -74,12 +73,12 @@ private[streaming] case class Data[T: ClassManifest](data: T) * his own Actor to run as receiver for Spark Streaming input source. * * This starts a supervisor actor which starts workers and also provides - * [http://doc.akka.io/docs/akka/2.0.5/scala/fault-tolerance.html fault-tolerance]. - * + * [http://doc.akka.io/docs/akka/2.0.5/scala/fault-tolerance.html fault-tolerance]. + * * Here's a way to start more supervisor/workers as its children. * * @example {{{ - * context.parent ! Props(new Supervisor) + * context.parent ! Props(new Supervisor) * }}} OR {{{ * context.parent ! Props(new Worker,"Worker") * }}} diff --git a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala index 5533c3cf1e..e7608f08ae 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala @@ -13,7 +13,7 @@ private[streaming] class ZeroMQReceiver[T: ClassManifest](publisherUrl: String, bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T]) extends Actor with Receiver with Logging { - override def preStart() = context.system.newSocket(SocketType.Sub, Listener(self), + override def preStart() = ZeroMQExtension(context.system).newSocket(SocketType.Sub, Listener(self), Connect(publisherUrl), subscribe) def receive: Receive = { -- cgit v1.2.3 From 8f3ac240cbdd678c0c76155b080dcc461355452e Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 29 Apr 2013 16:39:13 +0530 Subject: Fixed Warning: ClassManifest -> ClassTag --- core/src/main/scala/spark/PairRDDFunctions.scala | 31 ++++----- core/src/main/scala/spark/Partitioner.scala | 8 ++- core/src/main/scala/spark/RDD.scala | 43 ++++++------- core/src/main/scala/spark/RDDCheckpointData.scala | 6 +- .../scala/spark/SequenceFileRDDFunctions.scala | 9 +-- core/src/main/scala/spark/SparkContext.scala | 61 +++++++++--------- core/src/main/scala/spark/Utils.scala | 9 ++- .../main/scala/spark/api/java/JavaDoubleRDD.scala | 9 +-- .../main/scala/spark/api/java/JavaPairRDD.scala | 59 ++++++++--------- core/src/main/scala/spark/api/java/JavaRDD.scala | 14 ++--- .../main/scala/spark/api/java/JavaRDDLike.scala | 31 ++++----- .../scala/spark/api/java/JavaSparkContext.scala | 65 +++++++++---------- .../spark/api/java/function/FlatMapFunction.scala | 4 +- .../main/scala/spark/api/python/PythonRDD.scala | 7 ++- core/src/main/scala/spark/rdd/BlockRDD.scala | 4 +- core/src/main/scala/spark/rdd/CartesianRDD.scala | 5 +- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 3 +- core/src/main/scala/spark/rdd/CoalescedRDD.scala | 3 +- core/src/main/scala/spark/rdd/FilteredRDD.scala | 3 +- core/src/main/scala/spark/rdd/FlatMappedRDD.scala | 3 +- core/src/main/scala/spark/rdd/GlommedRDD.scala | 3 +- .../main/scala/spark/rdd/MapPartitionsRDD.scala | 3 +- .../spark/rdd/MapPartitionsWithIndexRDD.scala | 3 +- core/src/main/scala/spark/rdd/MappedRDD.scala | 3 +- .../scala/spark/rdd/ParallelCollectionRDD.scala | 8 ++- .../main/scala/spark/rdd/PartitionPruningRDD.scala | 5 +- core/src/main/scala/spark/rdd/PipedRDD.scala | 3 +- core/src/main/scala/spark/rdd/SampledRDD.scala | 5 +- core/src/main/scala/spark/rdd/SubtractedRDD.scala | 5 +- core/src/main/scala/spark/rdd/UnionRDD.scala | 5 +- core/src/main/scala/spark/rdd/ZippedRDD.scala | 8 ++- .../main/scala/spark/scheduler/DAGScheduler.scala | 9 +-- core/src/test/scala/spark/CheckpointSuite.scala | 5 +- .../spark/streaming/examples/ActorWordCount.scala | 3 +- .../src/main/scala/spark/streaming/DStream.scala | 27 ++++---- .../spark/streaming/DStreamCheckpointData.scala | 7 ++- .../spark/streaming/PairDStreamFunctions.scala | 43 +++++++------ .../scala/spark/streaming/StreamingContext.scala | 35 ++++++----- .../spark/streaming/api/java/JavaDStream.scala | 8 ++- .../spark/streaming/api/java/JavaDStreamLike.scala | 33 +++++----- .../spark/streaming/api/java/JavaPairDStream.scala | 57 ++++++++--------- .../streaming/api/java/JavaStreamingContext.scala | 73 +++++++++++----------- .../spark/streaming/dstream/CoGroupedDStream.scala | 4 +- .../streaming/dstream/ConstantInputDStream.scala | 6 +- .../spark/streaming/dstream/FileInputDStream.scala | 12 ++-- .../spark/streaming/dstream/FilteredDStream.scala | 6 +- .../streaming/dstream/FlatMapValuedDStream.scala | 4 +- .../streaming/dstream/FlatMappedDStream.scala | 5 +- .../streaming/dstream/FlumeInputDStream.scala | 3 +- .../spark/streaming/dstream/ForEachDStream.scala | 3 +- .../spark/streaming/dstream/GlommedDStream.scala | 4 +- .../spark/streaming/dstream/InputDStream.scala | 4 +- .../streaming/dstream/KafkaInputDStream.scala | 8 +-- .../streaming/dstream/MapPartitionedDStream.scala | 5 +- .../spark/streaming/dstream/MapValuedDStream.scala | 5 +- .../spark/streaming/dstream/MappedDStream.scala | 5 +- .../streaming/dstream/NetworkInputDStream.scala | 5 +- .../streaming/dstream/PluggableInputDStream.scala | 4 +- .../streaming/dstream/QueueInputDStream.scala | 12 ++-- .../spark/streaming/dstream/RawInputDStream.scala | 4 +- .../streaming/dstream/ReducedWindowedDStream.scala | 10 +-- .../spark/streaming/dstream/ShuffledDStream.scala | 4 +- .../streaming/dstream/SocketInputDStream.scala | 6 +- .../spark/streaming/dstream/StateDStream.scala | 4 +- .../streaming/dstream/TransformedDStream.scala | 4 +- .../spark/streaming/dstream/UnionDStream.scala | 6 +- .../spark/streaming/dstream/WindowedDStream.scala | 7 +-- .../spark/streaming/receivers/ActorReceiver.scala | 14 +++-- .../spark/streaming/receivers/ZeroMQReceiver.scala | 4 +- .../spark/streaming/util/MasterFailureTest.scala | 31 +++++---- .../test/java/spark/streaming/JavaTestUtils.scala | 22 ++++--- .../scala/spark/streaming/CheckpointSuite.scala | 5 +- .../test/scala/spark/streaming/TestSuiteBase.scala | 25 ++++---- 73 files changed, 531 insertions(+), 438 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 07efba9e8d..0453e98e7d 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -7,6 +7,7 @@ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ +import scala.reflect.{ ClassTag, classTag} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -29,7 +30,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]( +class PairRDDFunctions[K: ClassTag, V: ClassTag]( self: RDD[(K, V)]) extends Logging with HadoopMapReduceUtil @@ -394,7 +395,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( val cg = new CoGroupedRDD[K]( Seq(self.asInstanceOf[RDD[(K, _)]], other.asInstanceOf[RDD[(K, _)]]), partitioner) - val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest) + val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classTag[K], ClassTags.seqSeqClassTag) prfs.mapValues { case Seq(vs, ws) => (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]]) @@ -415,7 +416,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( other1.asInstanceOf[RDD[(K, _)]], other2.asInstanceOf[RDD[(K, _)]]), partitioner) - val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest) + val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classTag[K], ClassTags.seqSeqClassTag) prfs.mapValues { case Seq(vs, w1s, w2s) => (vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W1]], w2s.asInstanceOf[Seq[W2]]) @@ -469,19 +470,19 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Return an RDD with the pairs from `this` whose keys are not in `other`. - * + * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting * RDD will be <= us. */ - def subtractByKey[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, V)] = + def subtractByKey[W: ClassTag](other: RDD[(K, W)]): RDD[(K, V)] = subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.size))) /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ - def subtractByKey[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] = + def subtractByKey[W: ClassTag](other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] = subtractByKey(other, new HashPartitioner(numPartitions)) /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ - def subtractByKey[W: ClassManifest](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] = + def subtractByKey[W: ClassTag](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] = new SubtractedRDD[K, V, W](self, other, p) /** @@ -510,7 +511,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class * supporting the key and value types K and V in this RDD. */ - def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassManifest[F]) { + def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) } @@ -518,7 +519,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat` * (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD. */ - def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassManifest[F]) { + def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) } @@ -644,15 +645,15 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Return an RDD with the keys of each tuple. */ def keys: RDD[K] = self.map(_._1) - + /** * Return an RDD with the values of each tuple. */ def values: RDD[V] = self.map(_._2) - private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure + private[spark] def getKeyClass() = implicitly[ClassTag[K]].erasure - private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure + private[spark] def getValueClass() = implicitly[ClassTag[V]].erasure } /** @@ -660,7 +661,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * 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]( +class OrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag]( self: RDD[(K, V)]) extends Logging with Serializable { @@ -704,6 +705,6 @@ class FlatMappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => TraversableOnce[U] } } -private[spark] object Manifests { - val seqSeqManifest = classManifest[Seq[Seq[_]]] +private[spark] object ClassTags { + val seqSeqClassTag = classTag[Seq[Seq[_]]] } diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala index 6f8cd17c88..fc811d85e1 100644 --- a/core/src/main/scala/spark/Partitioner.scala +++ b/core/src/main/scala/spark/Partitioner.scala @@ -1,5 +1,7 @@ package spark +import scala.reflect.ClassTag + /** * 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`. @@ -60,7 +62,7 @@ class HashPartitioner(partitions: Int) extends Partitioner { } } } - + override def equals(other: Any): Boolean = other match { case h: HashPartitioner => h.numPartitions == numPartitions @@ -73,10 +75,10 @@ class HashPartitioner(partitions: Int) extends Partitioner { * A [[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]( +class RangePartitioner[K <% Ordered[K]: ClassTag, V]( partitions: Int, @transient rdd: RDD[(K,V)], - private val ascending: Boolean = true) + private val ascending: Boolean = true) extends Partitioner { // An array of upper bounds for the first (partitions - 1) partitions diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index ccd9d0364a..e6e0997a59 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -8,6 +8,7 @@ import scala.collection.Map import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap +import scala.reflect.{classTag, ClassTag} import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.NullWritable @@ -65,7 +66,7 @@ import SparkContext._ * [[http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details * on RDD internals. */ -abstract class RDD[T: ClassManifest]( +abstract class RDD[T: ClassTag]( @transient private var sc: SparkContext, @transient private var deps: Seq[Dependency[_]] ) extends Serializable with Logging { @@ -213,13 +214,13 @@ abstract class RDD[T: ClassManifest]( /** * Return a new RDD by applying a function to all elements of this RDD. */ - def map[U: ClassManifest](f: T => U): RDD[U] = new MappedRDD(this, sc.clean(f)) + def map[U: ClassTag](f: T => U): RDD[U] = new MappedRDD(this, sc.clean(f)) /** * Return a new RDD by first applying a function to all elements of this * RDD, and then flattening the results. */ - def flatMap[U: ClassManifest](f: T => TraversableOnce[U]): RDD[U] = + def flatMap[U: ClassTag](f: T => TraversableOnce[U]): RDD[U] = new FlatMappedRDD(this, sc.clean(f)) /** @@ -307,25 +308,25 @@ abstract class RDD[T: ClassManifest]( * Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of * elements (a, b) where a is in `this` and b is in `other`. */ - def cartesian[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other) + def cartesian[U: ClassTag](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other) /** * Return an RDD of grouped items. */ - def groupBy[K: ClassManifest](f: T => K): RDD[(K, Seq[T])] = + def groupBy[K: ClassTag](f: T => K): RDD[(K, Seq[T])] = groupBy[K](f, defaultPartitioner(this)) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K: ClassManifest](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] = + def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] = groupBy(f, new HashPartitioner(numPartitions)) /** * Return an RDD of grouped items. */ - def groupBy[K: ClassManifest](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = { + def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = { val cleanF = sc.clean(f) this.map(t => (cleanF(t), t)).groupByKey(p) } @@ -349,7 +350,7 @@ abstract class RDD[T: ClassManifest]( /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions[U: ClassManifest](f: Iterator[T] => Iterator[U], + def mapPartitions[U: ClassTag](f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = new MapPartitionsRDD(this, sc.clean(f), preservesPartitioning) @@ -357,7 +358,7 @@ abstract class RDD[T: ClassManifest]( * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. */ - def mapPartitionsWithIndex[U: ClassManifest]( + def mapPartitionsWithIndex[U: ClassTag]( f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning) @@ -367,7 +368,7 @@ abstract class RDD[T: ClassManifest]( * of the original partition. */ @deprecated("use mapPartitionsWithIndex", "0.7.0") - def mapPartitionsWithSplit[U: ClassManifest]( + def mapPartitionsWithSplit[U: ClassTag]( f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning) @@ -377,7 +378,7 @@ abstract class RDD[T: ClassManifest]( * additional parameter is produced by constructA, which is called in each * partition with the index of that partition. */ - def mapWith[A: ClassManifest, U: ClassManifest](constructA: Int => A, preservesPartitioning: Boolean = false) + def mapWith[A: ClassTag, U: ClassTag](constructA: Int => A, preservesPartitioning: Boolean = false) (f:(T, A) => U): RDD[U] = { def iterF(index: Int, iter: Iterator[T]): Iterator[U] = { val a = constructA(index) @@ -391,7 +392,7 @@ abstract class RDD[T: ClassManifest]( * additional parameter is produced by constructA, which is called in each * partition with the index of that partition. */ - def flatMapWith[A: ClassManifest, U: ClassManifest](constructA: Int => A, preservesPartitioning: Boolean = false) + def flatMapWith[A: ClassTag, U: ClassTag](constructA: Int => A, preservesPartitioning: Boolean = false) (f:(T, A) => Seq[U]): RDD[U] = { def iterF(index: Int, iter: Iterator[T]): Iterator[U] = { val a = constructA(index) @@ -405,7 +406,7 @@ abstract class RDD[T: ClassManifest]( * This additional parameter is produced by constructA, which is called in each * partition with the index of that partition. */ - def foreachWith[A: ClassManifest](constructA: Int => A) + def foreachWith[A: ClassTag](constructA: Int => A) (f:(T, A) => Unit) { def iterF(index: Int, iter: Iterator[T]): Iterator[T] = { val a = constructA(index) @@ -419,7 +420,7 @@ abstract class RDD[T: ClassManifest]( * additional parameter is produced by constructA, which is called in each * partition with the index of that partition. */ - def filterWith[A: ClassManifest](constructA: Int => A) + def filterWith[A: ClassTag](constructA: Int => A) (p:(T, A) => Boolean): RDD[T] = { def iterF(index: Int, iter: Iterator[T]): Iterator[T] = { val a = constructA(index) @@ -434,7 +435,7 @@ abstract class RDD[T: ClassManifest]( * partitions* and the *same number of elements in each partition* (e.g. one was made through * a map on the other). */ - def zip[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other) + def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other) // Actions (launch a job to return a value to the user program) @@ -470,7 +471,7 @@ abstract class RDD[T: ClassManifest]( /** * Return an RDD that contains all matching values by applying `f`. */ - def collect[U: ClassManifest](f: PartialFunction[T, U]): RDD[U] = { + def collect[U: ClassTag](f: PartialFunction[T, U]): RDD[U] = { filter(f.isDefinedAt).map(f) } @@ -560,7 +561,7 @@ abstract class RDD[T: ClassManifest]( * allowed to modify and return their first argument instead of creating a new U to avoid memory * allocation. */ - def aggregate[U: ClassManifest](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = { + def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = { // Clone the zero value since we will also be serializing it as part of tasks var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) val cleanSeqOp = sc.clean(seqOp) @@ -607,7 +608,7 @@ abstract class RDD[T: ClassManifest]( * combine step happens locally on the master, equivalent to running a single reduce task. */ def countByValue(): Map[T, Long] = { - if (elementClassManifest.erasure.isArray) { + if (elementClassTag.erasure.isArray) { throw new SparkException("countByValue() does not support arrays") } // TODO: This should perhaps be distributed by default. @@ -638,7 +639,7 @@ abstract class RDD[T: ClassManifest]( timeout: Long, confidence: Double = 0.95 ): PartialResult[Map[T, BoundedDouble]] = { - if (elementClassManifest.erasure.isArray) { + if (elementClassTag.erasure.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } val countPartition: (TaskContext, Iterator[T]) => OLMap[T] = { (ctx, iter) => @@ -751,12 +752,12 @@ abstract class RDD[T: ClassManifest]( /** Record user function generating this RDD. */ private[spark] val origin = Utils.getSparkCallSite - private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T] + private[spark] def elementClassTag: ClassTag[T] = classTag[T] private[spark] var checkpointData: Option[RDDCheckpointData[T]] = None /** Returns the first parent RDD */ - protected[spark] def firstParent[U: ClassManifest] = { + protected[spark] def firstParent[U: ClassTag] = { dependencies.head.rdd.asInstanceOf[RDD[U]] } diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/spark/RDDCheckpointData.scala index d00092e984..083ba9b8fa 100644 --- a/core/src/main/scala/spark/RDDCheckpointData.scala +++ b/core/src/main/scala/spark/RDDCheckpointData.scala @@ -1,7 +1,11 @@ package spark +import scala.reflect.ClassTag + import org.apache.hadoop.fs.Path + import rdd.{CheckpointRDD, CoalescedRDD} + import scheduler.{ResultTask, ShuffleMapTask} /** @@ -19,7 +23,7 @@ private[spark] object CheckpointState extends Enumeration { * manages the post-checkpoint state by providing the updated partitions, iterator and preferred locations * of the checkpointed RDD. */ -private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T]) +private[spark] class RDDCheckpointData[T: ClassTag](rdd: RDD[T]) extends Logging with Serializable { import CheckpointState._ diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala index 518034e07b..900d73bf42 100644 --- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala @@ -11,6 +11,7 @@ import java.util.Date import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.Map import scala.collection.mutable.HashMap +import scala.reflect.{ classTag, ClassTag} import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputFormat @@ -32,15 +33,15 @@ import spark.SparkContext._ * * Users should import `spark.SparkContext._` at the top of their program to use these functions. */ -class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : ClassManifest]( +class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag]( self: RDD[(K, V)]) extends Logging with Serializable { - private def getWritableClass[T <% Writable: ClassManifest](): Class[_ <: Writable] = { + private def getWritableClass[T <% Writable: ClassTag](): Class[_ <: Writable] = { val c = { - if (classOf[Writable].isAssignableFrom(classManifest[T].erasure)) { - classManifest[T].erasure + if (classOf[Writable].isAssignableFrom(classTag[T].erasure)) { + classTag[T].erasure } else { // We get the type of the Writable class by looking at the apply method which converts // from T to Writable. Since we have two apply methods we filter out the one which diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 4957a54c1b..6bd87bf3ec 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -8,6 +8,7 @@ import scala.collection.Map import scala.collection.generic.Growable import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ +import scala.reflect.{ ClassTag, classTag} import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration @@ -210,19 +211,19 @@ class SparkContext( // Methods for creating RDDs /** Distribute a local Scala collection to form an RDD. */ - def parallelize[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = { + def parallelize[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = { new ParallelCollectionRDD[T](this, seq, numSlices, Map[Int, Seq[String]]()) } /** Distribute a local Scala collection to form an RDD. */ - def makeRDD[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = { + def makeRDD[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = { parallelize(seq, numSlices) } /** Distribute a local Scala collection to form an RDD, with one or more * location preferences (hostnames of Spark nodes) for each object. * Create a new partition for each collection item. */ - def makeRDD[T: ClassManifest](seq: Seq[(T, Seq[String])]): RDD[T] = { + def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = { val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap new ParallelCollectionRDD[T](this, seq.map(_._1), seq.size, indexToPrefs) } @@ -265,7 +266,7 @@ class SparkContext( } /** - * Smarter version of hadoopFile() that uses class manifests to figure out the classes of keys, + * Smarter version of hadoopFile() that uses class tags to figure out the classes of keys, * values and the InputFormat so that users don't need to pass them directly. Instead, callers * can just write, for example, * {{{ @@ -273,7 +274,7 @@ class SparkContext( * }}} */ def hadoopFile[K, V, F <: InputFormat[K, V]](path: String, minSplits: Int) - (implicit km: ClassManifest[K], vm: ClassManifest[V], fm: ClassManifest[F]) + (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]) : RDD[(K, V)] = { hadoopFile(path, fm.erasure.asInstanceOf[Class[F]], @@ -283,7 +284,7 @@ class SparkContext( } /** - * Smarter version of hadoopFile() that uses class manifests to figure out the classes of keys, + * Smarter version of hadoopFile() that uses class tags to figure out the classes of keys, * values and the InputFormat so that users don't need to pass them directly. Instead, callers * can just write, for example, * {{{ @@ -291,12 +292,12 @@ class SparkContext( * }}} */ def hadoopFile[K, V, F <: InputFormat[K, V]](path: String) - (implicit km: ClassManifest[K], vm: ClassManifest[V], fm: ClassManifest[F]): RDD[(K, V)] = + (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = hadoopFile[K, V, F](path, defaultMinSplits) /** Get an RDD for a Hadoop file with an arbitrary new API InputFormat. */ def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]](path: String) - (implicit km: ClassManifest[K], vm: ClassManifest[V], fm: ClassManifest[F]): RDD[(K, V)] = { + (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = { newAPIHadoopFile( path, fm.erasure.asInstanceOf[Class[F]], @@ -359,11 +360,11 @@ class SparkContext( * IntWritable). The most natural thing would've been to have implicit objects for the * converters, but then we couldn't have an object for every subclass of Writable (you can't * have a parameterized singleton object). We use functions instead to create a new converter - * for the appropriate type. In addition, we pass the converter a ClassManifest of its type to + * for the appropriate type. In addition, we pass the converter a ClassTag of its type to * allow it to figure out the Writable class to use in the subclass case. */ def sequenceFile[K, V](path: String, minSplits: Int = defaultMinSplits) - (implicit km: ClassManifest[K], vm: ClassManifest[V], + (implicit km: ClassTag[K], vm: ClassTag[V], kcf: () => WritableConverter[K], vcf: () => WritableConverter[V]) : RDD[(K, V)] = { val kc = kcf() @@ -382,7 +383,7 @@ class SparkContext( * slow if you use the default serializer (Java serialization), though the nice thing about it is * that there's very little effort required to save arbitrary objects. */ - def objectFile[T: ClassManifest]( + def objectFile[T: ClassTag]( path: String, minSplits: Int = defaultMinSplits ): RDD[T] = { @@ -391,17 +392,17 @@ class SparkContext( } - protected[spark] def checkpointFile[T: ClassManifest]( + protected[spark] def checkpointFile[T: ClassTag]( path: String ): RDD[T] = { new CheckpointRDD[T](this, path) } /** Build the union of a list of RDDs. */ - def union[T: ClassManifest](rdds: Seq[RDD[T]]): RDD[T] = new UnionRDD(this, rdds) + def union[T: ClassTag](rdds: Seq[RDD[T]]): RDD[T] = new UnionRDD(this, rdds) /** Build the union of a list of RDDs passed as variable-length arguments. */ - def union[T: ClassManifest](first: RDD[T], rest: RDD[T]*): RDD[T] = + def union[T: ClassTag](first: RDD[T], rest: RDD[T]*): RDD[T] = new UnionRDD(this, Seq(first) ++ rest) // Methods for creating shared variables @@ -569,7 +570,7 @@ class SparkContext( * flag specifies whether the scheduler can run the computation on the driver rather than * shipping it out to the cluster, for short actions like first(). */ - def runJob[T, U: ClassManifest]( + def runJob[T, U: ClassTag]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], @@ -589,7 +590,7 @@ class SparkContext( * allowLocal flag specifies whether the scheduler can run the computation on the driver rather * than shipping it out to the cluster, for short actions like first(). */ - def runJob[T, U: ClassManifest]( + def runJob[T, U: ClassTag]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], @@ -604,7 +605,7 @@ class SparkContext( * Run a job on a given set of partitions of an RDD, but take a function of type * `Iterator[T] => U` instead of `(TaskContext, Iterator[T]) => U`. */ - def runJob[T, U: ClassManifest]( + def runJob[T, U: ClassTag]( rdd: RDD[T], func: Iterator[T] => U, partitions: Seq[Int], @@ -616,21 +617,21 @@ class SparkContext( /** * Run a job on all partitions in an RDD and return the results in an array. */ - def runJob[T, U: ClassManifest](rdd: RDD[T], func: (TaskContext, Iterator[T]) => U): Array[U] = { + def runJob[T, U: ClassTag](rdd: RDD[T], func: (TaskContext, Iterator[T]) => U): Array[U] = { runJob(rdd, func, 0 until rdd.partitions.size, false) } /** * Run a job on all partitions in an RDD and return the results in an array. */ - def runJob[T, U: ClassManifest](rdd: RDD[T], func: Iterator[T] => U): Array[U] = { + def runJob[T, U: ClassTag](rdd: RDD[T], func: Iterator[T] => U): Array[U] = { runJob(rdd, func, 0 until rdd.partitions.size, false) } /** * Run a job on all partitions in an RDD and pass the results to a handler function. */ - def runJob[T, U: ClassManifest]( + def runJob[T, U: ClassTag]( rdd: RDD[T], processPartition: (TaskContext, Iterator[T]) => U, resultHandler: (Int, U) => Unit) @@ -641,7 +642,7 @@ class SparkContext( /** * Run a job on all partitions in an RDD and pass the results to a handler function. */ - def runJob[T, U: ClassManifest]( + def runJob[T, U: ClassTag]( rdd: RDD[T], processPartition: Iterator[T] => U, resultHandler: (Int, U) => Unit) @@ -745,14 +746,14 @@ object SparkContext { // TODO: Add AccumulatorParams for other types, e.g. lists and strings - implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) = + implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = new PairRDDFunctions(rdd) - implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable: ClassManifest]( + implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( rdd: RDD[(K, V)]) = new SequenceFileRDDFunctions(rdd) - implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( + implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag]( rdd: RDD[(K, V)]) = new OrderedRDDFunctions(rdd) @@ -777,16 +778,16 @@ object SparkContext { implicit def stringToText(s: String) = new Text(s) - private implicit def arrayToArrayWritable[T <% Writable: ClassManifest](arr: Traversable[T]): ArrayWritable = { + private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T]): ArrayWritable = { def anyToWritable[U <% Writable](u: U): Writable = u - new ArrayWritable(classManifest[T].erasure.asInstanceOf[Class[Writable]], + new ArrayWritable(classTag[T].erasure.asInstanceOf[Class[Writable]], arr.map(x => anyToWritable(x)).toArray) } // Helper objects for converting common types to Writable - private def simpleWritableConverter[T, W <: Writable: ClassManifest](convert: W => T) = { - val wClass = classManifest[W].erasure.asInstanceOf[Class[W]] + private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) = { + val wClass = classTag[W].erasure.asInstanceOf[Class[W]] new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W])) } @@ -834,11 +835,11 @@ object SparkContext { /** * A class encapsulating how to convert some type T to Writable. It stores both the Writable class * corresponding to T (e.g. IntWritable for Int) and a function for doing the conversion. - * The getter for the writable class takes a ClassManifest[T] in case this is a generic object + * The getter for the writable class takes a ClassTag[T] in case this is a generic object * that doesn't know the type of T when it is created. This sounds strange but is necessary to * support converting subclasses of Writable to themselves (writableWritableConverter). */ private[spark] class WritableConverter[T]( - val writableClass: ClassManifest[T] => Class[_ <: Writable], + val writableClass: ClassTag[T] => Class[_ <: Writable], val convert: Writable => T) extends Serializable diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 81daacf958..cdccb8b336 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -4,14 +4,19 @@ import java.io._ import java.net._ import java.util.{Locale, Random, UUID} import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor} + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} + import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.io.Source +import scala.reflect.ClassTag +import scala.Some + import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder -import scala.Some + import spark.serializer.SerializerInstance /** @@ -207,7 +212,7 @@ private object Utils extends Logging { * result in a new collection. Unlike scala.util.Random.shuffle, this method * uses a local random number generator, avoiding inter-thread contention. */ - def randomize[T: ClassManifest](seq: TraversableOnce[T]): Seq[T] = { + def randomize[T: ClassTag](seq: TraversableOnce[T]): Seq[T] = { randomizeInPlace(seq.toArray) } diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala index 16692c0440..3e88a9ccc7 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala @@ -1,5 +1,6 @@ package spark.api.java +import scala.reflect.ClassTag import spark.RDD import spark.SparkContext.doubleRDDToDoubleRDDFunctions import spark.api.java.function.{Function => JFunction} @@ -11,7 +12,7 @@ import spark.Partitioner class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, JavaDoubleRDD] { - override val classManifest: ClassManifest[Double] = implicitly[ClassManifest[Double]] + override val classTag: ClassTag[Double] = implicitly[ClassTag[Double]] override val rdd: RDD[Double] = srdd.map(x => Double.valueOf(x)) @@ -25,7 +26,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ def cache(): JavaDoubleRDD = fromRDD(srdd.cache()) - /** + /** * Set this RDD's storage level to persist its values across operations after the first time * it is computed. Can only be called once on each RDD. */ @@ -65,7 +66,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav /** * Return an RDD with the elements from `this` that are not in `other`. - * + * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting * RDD will be <= us. */ @@ -123,7 +124,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav /** Return the approximate sum of the elements in this RDD. */ def sumApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = srdd.sumApprox(timeout, confidence) - + /** Return the approximate sum of the elements in this RDD. */ def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout) } diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 30084df4e2..89c6d05383 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -5,6 +5,7 @@ import java.util.Comparator import scala.Tuple2 import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputFormat @@ -23,13 +24,13 @@ import spark.Partitioner._ import spark.RDD import spark.SparkContext.rddToPairRDDFunctions -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]] { +class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K], + implicit val vClassTag: ClassTag[V]) extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] { override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd) - override val classManifest: ClassManifest[(K, V)] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]] + override val classTag: ClassTag[(K, V)] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K, V]]] import JavaPairRDD._ @@ -38,7 +39,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ def cache(): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.cache()) - /** + /** * Set this RDD's storage level to persist its values across operations after the first time * it is computed. Can only be called once on each RDD. */ @@ -94,14 +95,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif override def first(): (K, V) = rdd.first() // Pair RDD functions - + /** - * Generic function to combine the elements for each key using a custom set of aggregation - * functions. Turns a JavaPairRDD[(K, V)] into a result of type JavaPairRDD[(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, List[Int]). Users provide three + * Generic function to combine the elements for each key using a custom set of aggregation + * functions. Turns a JavaPairRDD[(K, V)] into a result of type JavaPairRDD[(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, List[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. @@ -113,8 +114,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif mergeValue: JFunction2[C, V, C], mergeCombiners: JFunction2[C, C, C], partitioner: Partitioner): JavaPairRDD[K, C] = { - implicit val cm: ClassManifest[C] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]] + implicit val cm: ClassTag[C] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] fromRDD(rdd.combineByKey( createCombiner, mergeValue, @@ -151,14 +152,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif /** Count the number of elements for each key, and return the result to the master as a Map. */ def countByKey(): java.util.Map[K, Long] = mapAsJavaMap(rdd.countByKey()) - /** + /** * (Experimental) Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ def countByKeyApprox(timeout: Long): PartialResult[java.util.Map[K, BoundedDouble]] = rdd.countByKeyApprox(timeout).map(mapAsJavaMap) - /** + /** * (Experimental) Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ @@ -214,7 +215,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif /** * Return an RDD with the elements from `this` that are not in `other`. - * + * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting * RDD will be <= us. */ @@ -271,15 +272,15 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif : JavaPairRDD[K, (Option[V], W)] = fromRDD(rdd.rightOuterJoin(other, partitioner)) - /** + /** * Simplified version of combineByKey that hash-partitions the resulting RDD using the existing * partitioner/parallelism level. */ def combineByKey[C](createCombiner: JFunction[V, C], mergeValue: JFunction2[C, V, C], mergeCombiners: JFunction2[C, C, C]): JavaPairRDD[K, C] = { - implicit val cm: ClassManifest[C] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]] + implicit val cm: ClassTag[C] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] fromRDD(combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(rdd))) } @@ -362,8 +363,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * this also retains the original RDD's partitioning. */ def mapValues[U](f: JFunction[V, U]): JavaPairRDD[K, U] = { - implicit val cm: ClassManifest[U] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]] + implicit val cm: ClassTag[U] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] fromRDD(rdd.mapValues(f)) } @@ -374,8 +375,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairRDD[K, U] = { import scala.collection.JavaConverters._ def fn = (x: V) => f.apply(x).asScala - implicit val cm: ClassManifest[U] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]] + implicit val cm: ClassTag[U] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] fromRDD(rdd.flatMapValues(fn)) } @@ -541,22 +542,22 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif } object JavaPairRDD { - def groupByResultToJava[K, T](rdd: RDD[(K, Seq[T])])(implicit kcm: ClassManifest[K], - vcm: ClassManifest[T]): RDD[(K, JList[T])] = + def groupByResultToJava[K, T](rdd: RDD[(K, Seq[T])])(implicit kcm: ClassTag[K], + vcm: ClassTag[T]): RDD[(K, JList[T])] = rddToPairRDDFunctions(rdd).mapValues(seqAsJavaList _) - def cogroupResultToJava[W, K, V](rdd: RDD[(K, (Seq[V], Seq[W]))])(implicit kcm: ClassManifest[K], - vcm: ClassManifest[V]): RDD[(K, (JList[V], JList[W]))] = rddToPairRDDFunctions(rdd).mapValues((x: (Seq[V], + def cogroupResultToJava[W, K, V](rdd: RDD[(K, (Seq[V], Seq[W]))])(implicit kcm: ClassTag[K], + vcm: ClassTag[V]): RDD[(K, (JList[V], JList[W]))] = rddToPairRDDFunctions(rdd).mapValues((x: (Seq[V], Seq[W])) => (seqAsJavaList(x._1), seqAsJavaList(x._2))) def cogroupResult2ToJava[W1, W2, K, V](rdd: RDD[(K, (Seq[V], Seq[W1], - Seq[W2]))])(implicit kcm: ClassManifest[K]) : RDD[(K, (JList[V], JList[W1], + Seq[W2]))])(implicit kcm: ClassTag[K]) : RDD[(K, (JList[V], JList[W1], JList[W2]))] = rddToPairRDDFunctions(rdd).mapValues( (x: (Seq[V], Seq[W1], Seq[W2])) => (seqAsJavaList(x._1), seqAsJavaList(x._2), seqAsJavaList(x._3))) - def fromRDD[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]): JavaPairRDD[K, V] = + def fromRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd) implicit def toRDD[K, V](rdd: JavaPairRDD[K, V]): RDD[(K, V)] = rdd.rdd diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index e29f1e5899..032506383c 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -1,10 +1,11 @@ package spark.api.java +import scala.reflect.ClassTag import spark._ import spark.api.java.function.{Function => JFunction} import spark.storage.StorageLevel -class JavaRDD[T](val rdd: RDD[T])(implicit val classManifest: ClassManifest[T]) extends +class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) extends JavaRDDLike[T, JavaRDD[T]] { override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd) @@ -14,7 +15,7 @@ JavaRDDLike[T, JavaRDD[T]] { /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ def cache(): JavaRDD[T] = wrapRDD(rdd.cache()) - /** + /** * Set this RDD's storage level to persist its values across operations after the first time * it is computed. Can only be called once on each RDD. */ @@ -31,7 +32,7 @@ JavaRDDLike[T, JavaRDD[T]] { * Return a new RDD containing the distinct elements in this RDD. */ def distinct(numPartitions: Int): JavaRDD[T] = wrapRDD(rdd.distinct(numPartitions)) - + /** * Return a new RDD containing only the elements that satisfy a predicate. */ @@ -54,7 +55,7 @@ JavaRDDLike[T, JavaRDD[T]] { */ def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaRDD[T] = wrapRDD(rdd.sample(withReplacement, fraction, seed)) - + /** * Return the union of this RDD and another one. Any identical elements will appear multiple * times (use `.distinct()` to eliminate them). @@ -63,7 +64,7 @@ JavaRDDLike[T, JavaRDD[T]] { /** * Return an RDD with the elements from `this` that are not in `other`. - * + * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting * RDD will be <= us. */ @@ -85,8 +86,7 @@ JavaRDDLike[T, JavaRDD[T]] { object JavaRDD { - implicit def fromRDD[T: ClassManifest](rdd: RDD[T]): JavaRDD[T] = new JavaRDD[T](rdd) + implicit def fromRDD[T: ClassTag](rdd: RDD[T]): JavaRDD[T] = new JavaRDD[T](rdd) implicit def toRDD[T](rdd: JavaRDD[T]): RDD[T] = rdd.rdd } - diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index d884529d7a..a6555081b3 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -3,6 +3,7 @@ package spark.api.java import java.util.{List => JList} import scala.Tuple2 import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import spark.{SparkContext, Partition, RDD, TaskContext} import spark.api.java.JavaPairRDD._ @@ -15,7 +16,7 @@ import com.google.common.base.Optional trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def wrapRDD(rdd: RDD[T]): This - implicit val classManifest: ClassManifest[T] + implicit val classTag: ClassTag[T] def rdd: RDD[T] @@ -57,7 +58,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by applying a function to all elements of this RDD. */ def map[K2, V2](f: PairFunction[T, K2, V2]): JavaPairRDD[K2, V2] = { - def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]] + def cm = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K2, V2]]] new JavaPairRDD(rdd.map(f)(cm))(f.keyType(), f.valueType()) } @@ -88,7 +89,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def flatMap[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairRDD[K2, V2] = { import scala.collection.JavaConverters._ def fn = (x: T) => f.apply(x).asScala - def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]] + def cm = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K2, V2]]] JavaPairRDD.fromRDD(rdd.flatMap(fn)(cm))(f.keyType(), f.valueType()) } @@ -128,18 +129,18 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * elements (a, b) where a is in `this` and b is in `other`. */ def cartesian[U](other: JavaRDDLike[U, _]): JavaPairRDD[T, U] = - JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classManifest))(classManifest, - other.classManifest) + JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classTag))(classTag, + other.classTag) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JList[T]] = { - implicit val kcm: ClassManifest[K] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]] - implicit val vcm: ClassManifest[JList[T]] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[JList[T]]] + implicit val kcm: ClassTag[K] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val vcm: ClassTag[JList[T]] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[JList[T]]] JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(f.returnType)))(kcm, vcm) } @@ -148,10 +149,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * mapping to that key. */ def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = { - implicit val kcm: ClassManifest[K] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]] - implicit val vcm: ClassManifest[JList[T]] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[JList[T]]] + implicit val kcm: ClassTag[K] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val vcm: ClassTag[JList[T]] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[JList[T]]] JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(f.returnType)))(kcm, vcm) } @@ -179,7 +180,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * a map on the other). */ def zip[U](other: JavaRDDLike[U, _]): JavaPairRDD[T, U] = { - JavaPairRDD.fromRDD(rdd.zip(other.rdd)(other.classManifest))(classManifest, other.classManifest) + JavaPairRDD.fromRDD(rdd.zip(other.rdd)(other.classTag))(classTag, other.classTag) } // Actions (launch a job to return a value to the user program) @@ -304,7 +305,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Creates tuples of the elements in this RDD by applying `f`. */ def keyBy[K](f: JFunction[T, K]): JavaPairRDD[K, T] = { - implicit val kcm: ClassManifest[K] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]] + implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] JavaPairRDD.fromRDD(rdd.keyBy(f)) } diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala index 5f18b1e15b..b633273eaf 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala @@ -4,6 +4,7 @@ import java.util.{Map => JMap} import scala.collection.JavaConversions import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.InputFormat @@ -63,8 +64,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork /** Distribute a local Scala collection to form an RDD. */ def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices) } @@ -75,10 +76,10 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork /** Distribute a local Scala collection to form an RDD. */ def parallelizePairs[K, V](list: java.util.List[Tuple2[K, V]], numSlices: Int) : JavaPairRDD[K, V] = { - implicit val kcm: ClassManifest[K] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]] - implicit val vcm: ClassManifest[V] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]] + implicit val kcm: ClassTag[K] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val vcm: ClassTag[V] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] JavaPairRDD.fromRDD(sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices)) } @@ -113,16 +114,16 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork valueClass: Class[V], minSplits: Int ): JavaPairRDD[K, V] = { - implicit val kcm = ClassManifest.fromClass(keyClass) - implicit val vcm = ClassManifest.fromClass(valueClass) + implicit val kcm: ClassTag[K] = ClassTag(keyClass) + implicit val vcm: ClassTag[V] = ClassTag(valueClass) new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass, minSplits)) } /**Get an RDD for a Hadoop SequenceFile. */ def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]): JavaPairRDD[K, V] = { - implicit val kcm = ClassManifest.fromClass(keyClass) - implicit val vcm = ClassManifest.fromClass(valueClass) + implicit val kcm: ClassTag[K] = ClassTag(keyClass) + implicit val vcm: ClassTag[V] = ClassTag(valueClass) new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass)) } @@ -134,8 +135,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * that there's very little effort required to save arbitrary objects. */ def objectFile[T](path: String, minSplits: Int): JavaRDD[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] sc.objectFile(path, minSplits)(cm) } @@ -147,8 +148,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * that there's very little effort required to save arbitrary objects. */ def objectFile[T](path: String): JavaRDD[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] sc.objectFile(path)(cm) } @@ -164,8 +165,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork valueClass: Class[V], minSplits: Int ): JavaPairRDD[K, V] = { - implicit val kcm = ClassManifest.fromClass(keyClass) - implicit val vcm = ClassManifest.fromClass(valueClass) + implicit val kcm: ClassTag[K] = ClassTag(keyClass) + implicit val vcm: ClassTag[V] = ClassTag(valueClass) new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minSplits)) } @@ -180,8 +181,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork keyClass: Class[K], valueClass: Class[V] ): JavaPairRDD[K, V] = { - implicit val kcm = ClassManifest.fromClass(keyClass) - implicit val vcm = ClassManifest.fromClass(valueClass) + implicit val kcm: ClassTag[K] = ClassTag(keyClass) + implicit val vcm: ClassTag[V] = ClassTag(valueClass) new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass)) } @@ -193,8 +194,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork valueClass: Class[V], minSplits: Int ): JavaPairRDD[K, V] = { - implicit val kcm = ClassManifest.fromClass(keyClass) - implicit val vcm = ClassManifest.fromClass(valueClass) + implicit val kcm: ClassTag[K] = ClassTag(keyClass) + implicit val vcm: ClassTag[V] = ClassTag(valueClass) new JavaPairRDD(sc.hadoopFile(path, inputFormatClass, keyClass, valueClass, minSplits)) } @@ -205,8 +206,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork keyClass: Class[K], valueClass: Class[V] ): JavaPairRDD[K, V] = { - implicit val kcm = ClassManifest.fromClass(keyClass) - implicit val vcm = ClassManifest.fromClass(valueClass) + implicit val kcm: ClassTag[K] = ClassTag(keyClass) + implicit val vcm: ClassTag[V] = ClassTag(valueClass) new JavaPairRDD(sc.hadoopFile(path, inputFormatClass, keyClass, valueClass)) } @@ -221,8 +222,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork kClass: Class[K], vClass: Class[V], conf: Configuration): JavaPairRDD[K, V] = { - implicit val kcm = ClassManifest.fromClass(kClass) - implicit val vcm = ClassManifest.fromClass(vClass) + implicit val kcm: ClassTag[K] = ClassTag(kClass) + implicit val vcm: ClassTag[V] = ClassTag(vClass) new JavaPairRDD(sc.newAPIHadoopFile(path, fClass, kClass, vClass, conf)) } @@ -235,15 +236,15 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork fClass: Class[F], kClass: Class[K], vClass: Class[V]): JavaPairRDD[K, V] = { - implicit val kcm = ClassManifest.fromClass(kClass) - implicit val vcm = ClassManifest.fromClass(vClass) + implicit val kcm: ClassTag[K] = ClassTag(kClass) + implicit val vcm: ClassTag[V] = ClassTag(vClass) new JavaPairRDD(sc.newAPIHadoopRDD(conf, fClass, kClass, vClass)) } /** Build the union of two or more RDDs. */ override def union[T](first: JavaRDD[T], rest: java.util.List[JavaRDD[T]]): JavaRDD[T] = { val rdds: Seq[RDD[T]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.rdd) - implicit val cm: ClassManifest[T] = first.classManifest + implicit val cm: ClassTag[T] = first.classTag sc.union(rdds)(cm) } @@ -251,9 +252,9 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork override def union[K, V](first: JavaPairRDD[K, V], rest: java.util.List[JavaPairRDD[K, V]]) : JavaPairRDD[K, V] = { val rdds: Seq[RDD[(K, V)]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.rdd) - implicit val cm: ClassManifest[(K, V)] = first.classManifest - implicit val kcm: ClassManifest[K] = first.kManifest - implicit val vcm: ClassManifest[V] = first.vManifest + implicit val cm: ClassTag[(K, V)] = first.classTag + implicit val kcm: ClassTag[K] = first.kClassTag + implicit val vcm: ClassTag[V] = first.vClassTag new JavaPairRDD(sc.union(rdds)(cm))(kcm, vcm) } @@ -386,8 +387,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork } protected def checkpointFile[T](path: String): JavaRDD[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] new JavaRDD(sc.checkpointFile(path)) } } diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala index e027cdacd3..e69480607e 100644 --- a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala +++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala @@ -1,5 +1,7 @@ package spark.api.java.function +import scala.reflect.ClassTag + /** * A function that returns zero or more output records from each input record. */ @@ -7,5 +9,5 @@ abstract class FlatMapFunction[T, R] extends Function[T, java.lang.Iterable[R]] @throws(classOf[Exception]) def call(x: T) : java.lang.Iterable[R] - def elementType() : ClassManifest[R] = ClassManifest.Any.asInstanceOf[ClassManifest[R]] + def elementType() : ClassTag[R] = ClassTag.Any.asInstanceOf[ClassTag[R]] } diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 9b4d54ab4e..82959a33eb 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -6,6 +6,7 @@ import java.util.{List => JList, ArrayList => JArrayList, Collections} import scala.collection.JavaConversions._ import scala.io.Source +import scala.reflect.ClassTag import spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import spark.broadcast.Broadcast @@ -13,7 +14,7 @@ import spark._ import spark.rdd.PipedRDD -private[spark] class PythonRDD[T: ClassManifest]( +private[spark] class PythonRDD[T: ClassTag]( parent: RDD[T], command: Seq[String], envVars: java.util.Map[String, String], @@ -251,7 +252,7 @@ private[spark] object PythonRDD { } def takePartition[T](rdd: RDD[T], partition: Int): Iterator[T] = { - implicit val cm : ClassManifest[T] = rdd.elementClassManifest + implicit val cm : ClassTag[T] = rdd.elementClassTag rdd.context.runJob(rdd, ((x: Iterator[T]) => x.toArray), Seq(partition), true).head.iterator } } @@ -277,7 +278,7 @@ private class BytesToString extends spark.api.java.function.Function[Array[Byte] */ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) extends AccumulatorParam[JList[Array[Byte]]] { - + 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/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala index 7348c4f15b..f44d37a91f 100644 --- a/core/src/main/scala/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/spark/rdd/BlockRDD.scala @@ -1,6 +1,7 @@ package spark.rdd import scala.collection.mutable.HashMap +import scala.reflect.ClassTag import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition { @@ -8,7 +9,7 @@ private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Pa } private[spark] -class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String]) +class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[String]) extends RDD[T](sc, Nil) { @transient lazy val locations_ = { @@ -37,4 +38,3 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St locations_(split.asInstanceOf[BlockRDDPartition].blockId) } - diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala index 38600b8be4..20a7487c48 100644 --- a/core/src/main/scala/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala @@ -1,6 +1,9 @@ package spark.rdd import java.io.{ObjectOutputStream, IOException} + +import scala.reflect.ClassTag + import spark._ @@ -26,7 +29,7 @@ class CartesianPartition( } private[spark] -class CartesianRDD[T: ClassManifest, U:ClassManifest]( +class CartesianRDD[T: ClassTag, U:ClassTag]( sc: SparkContext, var rdd1 : RDD[T], var rdd2 : RDD[U]) diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 5db77eb142..700a4160c8 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -1,5 +1,6 @@ package spark.rdd +import scala.reflect.ClassTag import spark._ import org.apache.hadoop.mapred.{FileInputFormat, SequenceFileInputFormat, JobConf, Reporter} import org.apache.hadoop.conf.Configuration @@ -15,7 +16,7 @@ private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {} * This RDD represents a RDD checkpoint file (similar to HadoopRDD). */ private[spark] -class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: String) +class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String) extends RDD[T](sc, Nil) { @transient val fs = new Path(checkpointPath).getFileSystem(sc.hadoopConfiguration) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 6d862c0c28..0ca678e4af 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -1,5 +1,6 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} @@ -26,7 +27,7 @@ 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. */ -class CoalescedRDD[T: ClassManifest]( +class CoalescedRDD[T: ClassTag]( @transient var prev: RDD[T], maxPartitions: Int) extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/spark/rdd/FilteredRDD.scala index c84ec39d21..eee72520d9 100644 --- a/core/src/main/scala/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/spark/rdd/FilteredRDD.scala @@ -1,8 +1,9 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{OneToOneDependency, RDD, Partition, TaskContext} -private[spark] class FilteredRDD[T: ClassManifest]( +private[spark] class FilteredRDD[T: ClassTag]( prev: RDD[T], f: T => Boolean) extends RDD[T](prev) { diff --git a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala index 8ebc778925..ab39ed6b5d 100644 --- a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala +++ b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala @@ -1,10 +1,11 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{RDD, Partition, TaskContext} private[spark] -class FlatMappedRDD[U: ClassManifest, T: ClassManifest]( +class FlatMappedRDD[U: ClassTag, T: ClassTag]( prev: RDD[T], f: T => TraversableOnce[U]) extends RDD[U](prev) { diff --git a/core/src/main/scala/spark/rdd/GlommedRDD.scala b/core/src/main/scala/spark/rdd/GlommedRDD.scala index e16c7ba881..8783aa2e86 100644 --- a/core/src/main/scala/spark/rdd/GlommedRDD.scala +++ b/core/src/main/scala/spark/rdd/GlommedRDD.scala @@ -1,8 +1,9 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{RDD, Partition, TaskContext} -private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T]) +private[spark] class GlommedRDD[T: ClassTag](prev: RDD[T]) extends RDD[Array[T]](prev) { override def getPartitions: Array[Partition] = firstParent[T].partitions diff --git a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala index d283c5b2bb..685eefa09e 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala @@ -1,10 +1,11 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{RDD, Partition, TaskContext} private[spark] -class MapPartitionsRDD[U: ClassManifest, T: ClassManifest]( +class MapPartitionsRDD[U: ClassTag, T: ClassTag]( prev: RDD[T], f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false) diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala index afb7504ba1..2a043c994b 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala @@ -1,5 +1,6 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{RDD, Partition, TaskContext} @@ -9,7 +10,7 @@ import spark.{RDD, Partition, TaskContext} * information such as the number of tuples in a partition. */ private[spark] -class MapPartitionsWithIndexRDD[U: ClassManifest, T: ClassManifest]( +class MapPartitionsWithIndexRDD[U: ClassTag, T: ClassTag]( prev: RDD[T], f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/spark/rdd/MappedRDD.scala index af07311b6d..79762e46a2 100644 --- a/core/src/main/scala/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/spark/rdd/MappedRDD.scala @@ -1,9 +1,10 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{RDD, Partition, TaskContext} private[spark] -class MappedRDD[U: ClassManifest, T: ClassManifest](prev: RDD[T], f: T => U) +class MappedRDD[U: ClassTag, T: ClassTag](prev: RDD[T], f: T => U) extends RDD[U](prev) { override def getPartitions: Array[Partition] = firstParent[T].partitions diff --git a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala index 07585a88ce..81dbcde227 100644 --- a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala @@ -3,9 +3,11 @@ package spark.rdd import scala.collection.immutable.NumericRange import scala.collection.mutable.ArrayBuffer import scala.collection.Map +import scala.reflect.ClassTag + import spark.{RDD, TaskContext, SparkContext, Partition} -private[spark] class ParallelCollectionPartition[T: ClassManifest]( +private[spark] class ParallelCollectionPartition[T: ClassTag]( val rddId: Long, val slice: Int, values: Seq[T]) @@ -23,7 +25,7 @@ private[spark] class ParallelCollectionPartition[T: ClassManifest]( override val index: Int = slice } -private[spark] class ParallelCollectionRDD[T: ClassManifest]( +private[spark] class ParallelCollectionRDD[T: ClassTag]( @transient sc: SparkContext, @transient data: Seq[T], numSlices: Int, @@ -53,7 +55,7 @@ private object ParallelCollectionRDD { * collections specially, encoding the slices as other Ranges to minimize memory cost. This makes * it efficient to run Spark over RDDs representing large sets of numbers. */ - def slice[T: ClassManifest](seq: Seq[T], numSlices: Int): Seq[Seq[T]] = { + def slice[T: ClassTag](seq: Seq[T], numSlices: Int): Seq[Seq[T]] = { if (numSlices < 1) { throw new IllegalArgumentException("Positive number of slices required") } diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 41ff62dd22..0afea33f8e 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -1,5 +1,6 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext} @@ -29,7 +30,7 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo * and the execution DAG has a filter on the key, we can avoid launching tasks * on partitions that don't have the range covering the key. */ -class PartitionPruningRDD[T: ClassManifest]( +class PartitionPruningRDD[T: ClassTag]( @transient prev: RDD[T], @transient partitionFilterFunc: Int => Boolean) extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) { @@ -49,6 +50,6 @@ object PartitionPruningRDD { * when its type T is not known at compile time. */ def create[T](rdd: RDD[T], partitionFilterFunc: Int => Boolean) = { - new PartitionPruningRDD[T](rdd, partitionFilterFunc)(rdd.elementClassManifest) + new PartitionPruningRDD[T](rdd, partitionFilterFunc)(rdd.elementClassTag) } } diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index 962a1b21ad..34d32eb85a 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -7,6 +7,7 @@ import scala.collection.Map import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.io.Source +import scala.reflect.ClassTag import spark.{RDD, SparkEnv, Partition, TaskContext} @@ -15,7 +16,7 @@ import spark.{RDD, SparkEnv, Partition, TaskContext} * An RDD that pipes the contents of each parent partition through an external command * (printing them one per line) and returns the output as a collection of strings. */ -class PipedRDD[T: ClassManifest]( +class PipedRDD[T: ClassTag]( prev: RDD[T], command: Seq[String], envVars: Map[String, String]) diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala index 243673f151..a503088f61 100644 --- a/core/src/main/scala/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/spark/rdd/SampledRDD.scala @@ -1,5 +1,6 @@ package spark.rdd +import scala.reflect.ClassTag import java.util.Random import cern.jet.random.Poisson @@ -12,9 +13,9 @@ class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition override val index: Int = prev.index } -class SampledRDD[T: ClassManifest]( +class SampledRDD[T: ClassTag]( prev: RDD[T], - withReplacement: Boolean, + withReplacement: Boolean, frac: Double, seed: Int) extends RDD[T](prev) { diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 481e03b349..5e56900b18 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -1,8 +1,11 @@ package spark.rdd import java.util.{HashMap => JHashMap} + import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + import spark.RDD import spark.Partitioner import spark.Dependency @@ -28,7 +31,7 @@ import spark.OneToOneDependency * you can use `rdd1`'s partitioner/partition size and not worry about running * out of memory because of the size of `rdd2`. */ -private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassManifest]( +private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( @transient var rdd1: RDD[(K, V)], @transient var rdd2: RDD[(K, W)], part: Partitioner) extends RDD[(K, V)](rdd1.context, Nil) { diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala index 2c52a67e22..b4b21c0aac 100644 --- a/core/src/main/scala/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/spark/rdd/UnionRDD.scala @@ -1,10 +1,11 @@ package spark.rdd import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag import 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) +private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int) extends Partition { var split: Partition = rdd.partitions(splitIndex) @@ -23,7 +24,7 @@ private[spark] class UnionPartition[T: ClassManifest](idx: Int, rdd: RDD[T], spl } } -class UnionRDD[T: ClassManifest]( +class UnionRDD[T: ClassTag]( sc: SparkContext, @transient var rdds: Seq[RDD[T]]) extends RDD[T](sc, Nil) { // Nil since we implement getDependencies diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index 35b0e06785..1b438cd505 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -1,10 +1,12 @@ package spark.rdd -import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} +import scala.reflect.ClassTag + +import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext} -private[spark] class ZippedPartition[T: ClassManifest, U: ClassManifest]( +private[spark] class ZippedPartition[T: ClassTag, U: ClassTag]( idx: Int, @transient rdd1: RDD[T], @transient rdd2: RDD[U] @@ -25,7 +27,7 @@ private[spark] class ZippedPartition[T: ClassManifest, U: ClassManifest]( } } -class ZippedRDD[T: ClassManifest, U: ClassManifest]( +class ZippedRDD[T: ClassTag, U: ClassTag]( sc: SparkContext, var rdd1: RDD[T], var rdd2: RDD[U]) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index c54dce51d7..b838cf84a8 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -6,6 +6,7 @@ import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.TimeUnit import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} +import scala.reflect.ClassTag import spark._ import spark.executor.TaskMetrics @@ -215,7 +216,7 @@ class DAGScheduler( * The job is assumed to have at least one partition; zero partition jobs should be handled * without a JobSubmitted event. */ - private[scheduler] def prepareJob[T, U: ClassManifest]( + private[scheduler] def prepareJob[T, U: ClassTag]( finalRdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], @@ -231,7 +232,7 @@ class DAGScheduler( return (toSubmit, waiter) } - def runJob[T, U: ClassManifest]( + def runJob[T, U: ClassTag]( finalRdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], @@ -326,7 +327,7 @@ class DAGScheduler( submitStage(stage) } } - + /** * Check for waiting or failed stages which are now eligible for resubmission. * Ordinarily run on every iteration of the event loop. @@ -712,7 +713,7 @@ class DAGScheduler( sizeBefore = shuffleToMapStage.size shuffleToMapStage.clearOldValues(cleanupTime) logInfo("shuffleToMapStage " + sizeBefore + " --> " + shuffleToMapStage.size) - + sizeBefore = pendingTasks.size pendingTasks.clearOldValues(cleanupTime) logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size) diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index ca385972fb..8836c68ae6 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -1,5 +1,6 @@ package spark +import scala.reflect.ClassTag import org.scalatest.FunSuite import java.io.File import spark.rdd._ @@ -179,7 +180,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { * not, but this is not done by default as usually the partitions do not refer to any RDD and * therefore never store the lineage. */ - def testCheckpointing[U: ClassManifest]( + def testCheckpointing[U: ClassTag]( op: (RDD[Int]) => RDD[U], testRDDSize: Boolean = true, testRDDPartitionSize: Boolean = false @@ -248,7 +249,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { * RDDs partitions. So even if the parent RDD is checkpointed and its partitions changed, * this RDD will remember the partitions and therefore potentially the whole lineage. */ - def testParentCheckpointing[U: ClassManifest]( + def testParentCheckpointing[U: ClassTag]( op: (RDD[Int]) => RDD[U], testRDDSize: Boolean, testRDDPartitionSize: Boolean diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala index 3b847fe603..deb1c8511a 100644 --- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala @@ -2,6 +2,7 @@ package spark.streaming.examples import scala.collection.mutable.LinkedList import scala.util.Random +import scala.reflect.ClassTag import akka.actor.Actor import akka.actor.ActorRef @@ -65,7 +66,7 @@ class FeederActor extends Actor { * * @see [[spark.streaming.examples.FeederActor]] */ -class SampleActorReceiver[T: ClassManifest](urlOfPublisher: String) +class SampleActorReceiver[T: ClassTag](urlOfPublisher: String) extends Actor with Receiver { lazy private val remotePublisher = context.actorFor(urlOfPublisher) diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index e1be5ef51c..c307c69611 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -9,6 +9,7 @@ import spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap +import scala.reflect.ClassTag import java.io.{ObjectInputStream, IOException, ObjectOutputStream} @@ -36,7 +37,7 @@ import org.apache.hadoop.conf.Configuration * - A function that is used to generate an RDD after each time interval */ -abstract class DStream[T: ClassManifest] ( +abstract class DStream[T: ClassTag] ( @transient protected[streaming] var ssc: StreamingContext ) extends Serializable with Logging { @@ -62,7 +63,7 @@ abstract class DStream[T: ClassManifest] ( // RDDs generated, marked as protected[streaming] so that testsuites can access it @transient protected[streaming] var generatedRDDs = new HashMap[Time, RDD[T]] () - + // Time zero for the DStream protected[streaming] var zeroTime: Time = null @@ -254,16 +255,16 @@ abstract class DStream[T: ClassManifest] ( /** * Retrieve a precomputed RDD of this DStream, or computes the RDD. This is an internal * method that should not be called directly. - */ + */ protected[streaming] def getOrCompute(time: Time): Option[RDD[T]] = { // If this DStream was not initialized (i.e., zeroTime not set), then do it // If RDD was already generated, then retrieve it from HashMap generatedRDDs.get(time) match { - - // If an RDD was already generated and is being reused, then + + // If an RDD was already generated and is being reused, then // probably all RDDs in this DStream will be reused and hence should be cached case Some(oldRDD) => Some(oldRDD) - + // if RDD was not generated, and if the time is valid // (based on sliding time of this DStream), then generate the RDD case None => { @@ -280,7 +281,7 @@ abstract class DStream[T: ClassManifest] ( } generatedRDDs.put(time, newRDD) Some(newRDD) - case None => + case None => None } } else { @@ -324,7 +325,7 @@ abstract class DStream[T: ClassManifest] ( dependencies.foreach(_.clearOldMetadata(time)) } - /* Adds metadata to the Stream while it is running. + /* Adds metadata to the Stream while it is running. * This methd should be overwritten by sublcasses of InputDStream. */ protected[streaming] def addMetadata(metadata: Any) { @@ -396,7 +397,7 @@ abstract class DStream[T: ClassManifest] ( // ======================================================================= /** Return a new DStream by applying a function to all elements of this DStream. */ - def map[U: ClassManifest](mapFunc: T => U): DStream[U] = { + def map[U: ClassTag](mapFunc: T => U): DStream[U] = { new MappedDStream(this, context.sparkContext.clean(mapFunc)) } @@ -404,7 +405,7 @@ abstract class DStream[T: ClassManifest] ( * Return a new DStream by applying a function to all elements of this DStream, * and then flattening the results */ - def flatMap[U: ClassManifest](flatMapFunc: T => Traversable[U]): DStream[U] = { + def flatMap[U: ClassTag](flatMapFunc: T => Traversable[U]): DStream[U] = { new FlatMappedDStream(this, context.sparkContext.clean(flatMapFunc)) } @@ -423,7 +424,7 @@ abstract class DStream[T: ClassManifest] ( * of this DStream. Applying mapPartitions() to an RDD applies a function to each partition * of the RDD. */ - def mapPartitions[U: ClassManifest]( + def mapPartitions[U: ClassTag]( mapPartFunc: Iterator[T] => Iterator[U], preservePartitioning: Boolean = false ): DStream[U] = { @@ -474,7 +475,7 @@ abstract class DStream[T: ClassManifest] ( * Return a new DStream in which each RDD is generated by applying a function * on each RDD of this DStream. */ - def transform[U: ClassManifest](transformFunc: RDD[T] => RDD[U]): DStream[U] = { + def transform[U: ClassTag](transformFunc: RDD[T] => RDD[U]): DStream[U] = { transform((r: RDD[T], t: Time) => transformFunc(r)) } @@ -482,7 +483,7 @@ abstract class DStream[T: ClassManifest] ( * Return a new DStream in which each RDD is generated by applying a function * on each RDD of this DStream. */ - def transform[U: ClassManifest](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = { + def transform[U: ClassTag](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = { new TransformedDStream(this, context.sparkContext.clean(transformFunc)) } diff --git a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala index 6b0fade7c6..82e5bb6e49 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala @@ -3,13 +3,15 @@ package 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 scala.collection.mutable.HashMap +import scala.reflect.ClassTag private[streaming] -class DStreamCheckpointData[T: ClassManifest] (dstream: DStream[T]) +class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) extends Serializable with Logging { protected val data = new HashMap[Time, AnyRef]() @@ -90,4 +92,3 @@ class DStreamCheckpointData[T: ClassManifest] (dstream: DStream[T]) "[\n" + checkpointFiles.size + " checkpoint files \n" + checkpointFiles.mkString("\n") + "\n]" } } - diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index 3ec922957d..962ba6619d 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -5,18 +5,19 @@ import spark.streaming.dstream.{ReducedWindowedDStream, StateDStream} import spark.streaming.dstream.{CoGroupedDStream, ShuffledDStream} import spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream} -import spark.{Manifests, RDD, Partitioner, HashPartitioner} +import spark.{ClassTags, RDD, Partitioner, HashPartitioner} import spark.SparkContext._ import spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer +import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.mapred.{JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.hadoop.mapred.OutputFormat import org.apache.hadoop.conf.Configuration -class PairDStreamFunctions[K: ClassManifest, V: ClassManifest](self: DStream[(K,V)]) +class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) extends Serializable { private[streaming] def ssc = self.ssc @@ -86,7 +87,7 @@ extends Serializable { * combineByKey for RDDs. Please refer to combineByKey in [[spark.PairRDDFunctions]] for more * information. */ - def combineByKey[C: ClassManifest]( + def combineByKey[C: ClassTag]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiner: (C, C) => C, @@ -186,7 +187,7 @@ extends Serializable { * DStream's batching interval */ def reduceByKeyAndWindow( - reduceFunc: (V, V) => V, + reduceFunc: (V, V) => V, windowDuration: Duration, slideDuration: Duration ): DStream[(K, V)] = { @@ -317,7 +318,7 @@ extends Serializable { * corresponding state key-value pair will be eliminated. * @tparam S State type */ - def updateStateByKey[S: ClassManifest]( + def updateStateByKey[S: ClassTag]( updateFunc: (Seq[V], Option[S]) => Option[S] ): DStream[(K, S)] = { updateStateByKey(updateFunc, defaultPartitioner()) @@ -332,7 +333,7 @@ extends Serializable { * @param numPartitions Number of partitions of each RDD in the new DStream. * @tparam S State type */ - def updateStateByKey[S: ClassManifest]( + def updateStateByKey[S: ClassTag]( updateFunc: (Seq[V], Option[S]) => Option[S], numPartitions: Int ): DStream[(K, S)] = { @@ -348,7 +349,7 @@ extends Serializable { * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream. * @tparam S State type */ - def updateStateByKey[S: ClassManifest]( + def updateStateByKey[S: ClassTag]( updateFunc: (Seq[V], Option[S]) => Option[S], partitioner: Partitioner ): DStream[(K, S)] = { @@ -371,7 +372,7 @@ extends Serializable { * @param rememberPartitioner Whether to remember the paritioner object in the generated RDDs. * @tparam S State type */ - def updateStateByKey[S: ClassManifest]( + def updateStateByKey[S: ClassTag]( updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)], partitioner: Partitioner, rememberPartitioner: Boolean @@ -380,11 +381,11 @@ extends Serializable { } - def mapValues[U: ClassManifest](mapValuesFunc: V => U): DStream[(K, U)] = { + def mapValues[U: ClassTag](mapValuesFunc: V => U): DStream[(K, U)] = { new MapValuedDStream[K, V, U](self, mapValuesFunc) } - def flatMapValues[U: ClassManifest]( + def flatMapValues[U: ClassTag]( flatMapValuesFunc: V => TraversableOnce[U] ): DStream[(K, U)] = { new FlatMapValuedDStream[K, V, U](self, flatMapValuesFunc) @@ -396,7 +397,7 @@ extends Serializable { * key in both RDDs. HashPartitioner is used to partition each generated RDD into default number * of partitions. */ - def cogroup[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = { cogroup(other, defaultPartitioner()) } @@ -405,7 +406,7 @@ extends Serializable { * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that * key in both RDDs. Partitioner is used to partition each generated RDD. */ - def cogroup[W: ClassManifest]( + def cogroup[W: ClassTag]( other: DStream[(K, W)], partitioner: Partitioner ): DStream[(K, (Seq[V], Seq[W]))] = { @@ -415,8 +416,8 @@ extends Serializable { partitioner ) val pdfs = new PairDStreamFunctions[K, Seq[Seq[_]]](cgd)( - classManifest[K], - Manifests.seqSeqManifest + classTag[K], + ClassTags.seqSeqClassTag ) pdfs.mapValues { case Seq(vs, ws) => @@ -428,7 +429,7 @@ extends Serializable { * Join `this` DStream with `other` DStream. HashPartitioner is used * to partition each generated RDD into default number of partitions. */ - def join[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (V, W))] = { + def join[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (V, W))] = { join[W](other, defaultPartitioner()) } @@ -437,7 +438,7 @@ extends Serializable { * be generated by joining RDDs from `this` and other DStream. Uses the given * Partitioner to partition each generated RDD. */ - def join[W: ClassManifest]( + def join[W: ClassTag]( other: DStream[(K, W)], partitioner: Partitioner ): DStream[(K, (V, W))] = { @@ -455,7 +456,7 @@ extends Serializable { def saveAsHadoopFiles[F <: OutputFormat[K, V]]( prefix: String, suffix: String - )(implicit fm: ClassManifest[F]) { + )(implicit fm: ClassTag[F]) { saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) } @@ -485,7 +486,7 @@ extends Serializable { def saveAsNewAPIHadoopFiles[F <: NewOutputFormat[K, V]]( prefix: String, suffix: String - )(implicit fm: ClassManifest[F]) { + )(implicit fm: ClassTag[F]) { saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) } @@ -508,9 +509,7 @@ extends Serializable { self.foreach(saveFunc) } - private def getKeyClass() = implicitly[ClassManifest[K]].erasure + private def getKeyClass() = implicitly[ClassTag[K]].erasure - private def getValueClass() = implicitly[ClassManifest[V]].erasure + private def getValueClass() = implicitly[ClassTag[V]].erasure } - - diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index b8b60aab43..7646e15521 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -16,6 +16,7 @@ import spark.streaming.receivers.ActorReceiver import scala.collection.mutable.Queue import scala.collection.Map +import scala.reflect.ClassTag import java.io.InputStream import java.util.concurrent.atomic.AtomicInteger @@ -166,7 +167,7 @@ class StreamingContext private ( * Create an input stream with any arbitrary user implemented network receiver. * @param receiver Custom implementation of NetworkReceiver */ - def networkStream[T: ClassManifest]( + def networkStream[T: ClassTag]( receiver: NetworkReceiver[T]): DStream[T] = { val inputStream = new PluggableInputDStream[T](this, receiver) @@ -185,7 +186,7 @@ class StreamingContext private ( * to ensure the type safety, i.e parametrized type of data received and actorStream * should be same. */ - def actorStream[T: ClassManifest]( + def actorStream[T: ClassTag]( props: Props, name: String, storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2, @@ -203,7 +204,7 @@ class StreamingContext private ( * and sub sequence refer to its payload. * @param storageLevel RDD storage level. Defaults to memory-only. */ - def zeroMQStream[T: ClassManifest]( + def zeroMQStream[T: ClassTag]( publisherUrl:String, subscribe: Subscribe, bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T], @@ -225,7 +226,7 @@ class StreamingContext private ( * @param storageLevel Storage level to use for storing the received objects * (default: StorageLevel.MEMORY_AND_DISK_SER_2) */ - def kafkaStream[T: ClassManifest]( + def kafkaStream[T: ClassTag]( zkQuorum: String, groupId: String, topics: Map[String, Int], @@ -264,7 +265,7 @@ class StreamingContext private ( * @param storageLevel Storage level to use for storing the received objects * @tparam T Type of the objects received (after converting bytes to objects) */ - def socketStream[T: ClassManifest]( + def socketStream[T: ClassTag]( hostname: String, port: Int, converter: (InputStream) => Iterator[T], @@ -286,7 +287,7 @@ class StreamingContext private ( port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): DStream[SparkFlumeEvent] = { - val inputStream = new FlumeInputDStream(this, hostname, port, storageLevel) + val inputStream = new FlumeInputDStream[SparkFlumeEvent](this, hostname, port, storageLevel) registerInputStream(inputStream) inputStream } @@ -301,7 +302,7 @@ class StreamingContext private ( * @param storageLevel Storage level to use for storing the received objects * @tparam T Type of the objects in the received blocks */ - def rawSocketStream[T: ClassManifest]( + def rawSocketStream[T: ClassTag]( hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 @@ -321,9 +322,9 @@ class StreamingContext private ( * @tparam F Input format for reading HDFS file */ def fileStream[ - K: ClassManifest, - V: ClassManifest, - F <: NewInputFormat[K, V]: ClassManifest + K: ClassTag, + V: ClassTag, + F <: NewInputFormat[K, V]: ClassTag ] (directory: String): DStream[(K, V)] = { val inputStream = new FileInputDStream[K, V, F](this, directory) registerInputStream(inputStream) @@ -341,9 +342,9 @@ class StreamingContext private ( * @tparam F Input format for reading HDFS file */ def fileStream[ - K: ClassManifest, - V: ClassManifest, - F <: NewInputFormat[K, V]: ClassManifest + K: ClassTag, + V: ClassTag, + F <: NewInputFormat[K, V]: ClassTag ] (directory: String, filter: Path => Boolean, newFilesOnly: Boolean): DStream[(K, V)] = { val inputStream = new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly) registerInputStream(inputStream) @@ -385,7 +386,7 @@ class StreamingContext private ( * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval * @tparam T Type of objects in the RDD */ - def queueStream[T: ClassManifest]( + def queueStream[T: ClassTag]( queue: Queue[RDD[T]], oneAtATime: Boolean = true ): DStream[T] = { @@ -400,7 +401,7 @@ class StreamingContext private ( * @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]( + def queueStream[T: ClassTag]( queue: Queue[RDD[T]], oneAtATime: Boolean, defaultRDD: RDD[T] @@ -413,7 +414,7 @@ class StreamingContext private ( /** * Create a unified DStream from multiple DStreams of the same type and same interval */ - def union[T: ClassManifest](streams: Seq[DStream[T]]): DStream[T] = { + def union[T: ClassTag](streams: Seq[DStream[T]]): DStream[T] = { new UnionDStream[T](streams.toArray) } @@ -490,7 +491,7 @@ class StreamingContext private ( object StreamingContext { - implicit def toPairDStreamFunctions[K: ClassManifest, V: ClassManifest](stream: DStream[(K,V)]) = { + implicit def toPairDStreamFunctions[K: ClassTag, V: ClassTag](stream: DStream[(K,V)]) = { new PairDStreamFunctions[K, V](stream) } diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala index 4d93f0a5f7..535a302d60 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala @@ -6,6 +6,8 @@ import spark.api.java.JavaRDD import spark.storage.StorageLevel import spark.RDD +import scala.reflect.ClassTag + /** * 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]] @@ -24,7 +26,7 @@ import spark.RDD * - A time interval at which the DStream generates an RDD * - A function that is used to generate an RDD after each time interval */ -class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassManifest[T]) +class JavaDStream[T](val dstream: DStream[T])(implicit val classTag: ClassTag[T]) extends JavaDStreamLike[T, JavaDStream[T], JavaRDD[T]] { override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd) @@ -80,6 +82,6 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassM } object JavaDStream { - implicit def fromDStream[T: ClassManifest](dstream: DStream[T]): JavaDStream[T] = + implicit def fromDStream[T: ClassTag](dstream: DStream[T]): JavaDStream[T] = new JavaDStream[T](dstream) -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala index 548809a359..d67dd34760 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala @@ -4,6 +4,7 @@ import java.util.{List => JList} import java.lang.{Long => JLong} import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import spark.streaming._ import spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD} @@ -14,7 +15,7 @@ import JavaDStream._ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]] extends Serializable { - implicit val classManifest: ClassManifest[T] + implicit val classTag: ClassTag[T] def dstream: DStream[T] @@ -116,7 +117,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T /** Return a new DStream by applying a function to all elements of this DStream. */ def map[K2, V2](f: PairFunction[T, K2, V2]): JavaPairDStream[K2, V2] = { - def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]] + def cm = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K2, V2]]] new JavaPairDStream(dstream.map(f)(cm))(f.keyType(), f.valueType()) } @@ -137,7 +138,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T def flatMap[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairDStream[K2, V2] = { import scala.collection.JavaConverters._ def fn = (x: T) => f.apply(x).asScala - def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]] + def cm = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K2, V2]]] new JavaPairDStream(dstream.flatMap(fn)(cm))(f.keyType(), f.valueType()) } @@ -240,8 +241,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * on each RDD of this DStream. */ def transform[U](transformFunc: JFunction[R, JavaRDD[U]]): JavaDStream[U] = { - implicit val cm: ClassManifest[U] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]] + implicit val cm: ClassTag[U] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] def scalaTransform (in: RDD[T]): RDD[U] = transformFunc.call(wrapRDD(in)).rdd dstream.transform(scalaTransform(_)) @@ -252,8 +253,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * on each RDD of this DStream. */ def transform[U](transformFunc: JFunction2[R, Time, JavaRDD[U]]): JavaDStream[U] = { - implicit val cm: ClassManifest[U] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]] + implicit val cm: ClassTag[U] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] def scalaTransform (in: RDD[T], time: Time): RDD[U] = transformFunc.call(wrapRDD(in), time).rdd dstream.transform(scalaTransform(_, _)) @@ -265,10 +266,10 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T */ def transform[K2, V2](transformFunc: JFunction[R, JavaPairRDD[K2, V2]]): JavaPairDStream[K2, V2] = { - implicit val cmk: ClassManifest[K2] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]] - implicit val cmv: ClassManifest[V2] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]] + implicit val cmk: ClassTag[K2] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]] + implicit val cmv: ClassTag[V2] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]] def scalaTransform (in: RDD[T]): RDD[(K2, V2)] = transformFunc.call(wrapRDD(in)).rdd dstream.transform(scalaTransform(_)) @@ -280,10 +281,10 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T */ def transform[K2, V2](transformFunc: JFunction2[R, Time, JavaPairRDD[K2, V2]]): JavaPairDStream[K2, V2] = { - implicit val cmk: ClassManifest[K2] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]] - implicit val cmv: ClassManifest[V2] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]] + implicit val cmk: ClassTag[K2] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]] + implicit val cmv: ClassTag[V2] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]] def scalaTransform (in: RDD[T], time: Time): RDD[(K2, V2)] = transformFunc.call(wrapRDD(in), time).rdd dstream.transform(scalaTransform(_, _)) @@ -296,4 +297,4 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T def checkpoint(interval: Duration) = { dstream.checkpoint(interval) } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala index 30240cad98..4c93a7e440 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala @@ -4,6 +4,7 @@ import java.util.{List => JList} import java.lang.{Long => JLong} import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import spark.streaming._ import spark.streaming.StreamingContext._ @@ -18,8 +19,8 @@ import com.google.common.base.Optional import spark.RDD class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( - implicit val kManifiest: ClassManifest[K], - implicit val vManifest: ClassManifest[V]) + implicit val kTag: ClassTag[K], + implicit val vTag: ClassTag[V]) extends JavaDStreamLike[(K, V), JavaPairDStream[K, V], JavaPairRDD[K, V]] { override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd) @@ -138,8 +139,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( mergeCombiners: JFunction2[C, C, C], partitioner: Partitioner ): JavaPairDStream[K, C] = { - implicit val cm: ClassManifest[C] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]] + implicit val cm: ClassTag[C] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] dstream.combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner) } @@ -407,8 +408,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( */ def updateStateByKey[S](updateFunc: JFunction2[JList[V], Optional[S], Optional[S]]) : JavaPairDStream[K, S] = { - implicit val cm: ClassManifest[S] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[S]] + implicit val cm: ClassTag[S] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[S]] dstream.updateStateByKey(convertUpdateStateFunction(updateFunc)) } @@ -421,7 +422,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param numPartitions Number of partitions of each RDD in the new DStream. * @tparam S State type */ - def updateStateByKey[S: ClassManifest]( + def updateStateByKey[S: ClassTag]( updateFunc: JFunction2[JList[V], Optional[S], Optional[S]], numPartitions: Int) : JavaPairDStream[K, S] = { @@ -437,7 +438,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream. * @tparam S State type */ - def updateStateByKey[S: ClassManifest]( + def updateStateByKey[S: ClassTag]( updateFunc: JFunction2[JList[V], Optional[S], Optional[S]], partitioner: Partitioner ): JavaPairDStream[K, S] = { @@ -445,16 +446,16 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } def mapValues[U](f: JFunction[V, U]): JavaPairDStream[K, U] = { - implicit val cm: ClassManifest[U] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]] + implicit val cm: ClassTag[U] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] dstream.mapValues(f) } def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairDStream[K, U] = { import scala.collection.JavaConverters._ def fn = (x: V) => f.apply(x).asScala - implicit val cm: ClassManifest[U] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]] + implicit val cm: ClassTag[U] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] dstream.flatMapValues(fn) } @@ -465,8 +466,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * of partitions. */ def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JList[V], JList[W])] = { - implicit val cm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + implicit val cm: ClassTag[W] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] dstream.cogroup(other.dstream).mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) } @@ -477,8 +478,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( */ def cogroup[W](other: JavaPairDStream[K, W], partitioner: Partitioner) : JavaPairDStream[K, (JList[V], JList[W])] = { - implicit val cm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + implicit val cm: ClassTag[W] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] dstream.cogroup(other.dstream, partitioner) .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) } @@ -488,8 +489,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * to partition each generated RDD into default number of partitions. */ def join[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (V, W)] = { - implicit val cm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + implicit val cm: ClassTag[W] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] dstream.join(other.dstream) } @@ -500,8 +501,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( */ def join[W](other: JavaPairDStream[K, W], partitioner: Partitioner) : JavaPairDStream[K, (V, W)] = { - implicit val cm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + implicit val cm: ClassTag[W] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] dstream.join(other.dstream, partitioner) } @@ -575,24 +576,24 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( dstream.saveAsNewAPIHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass, conf) } - override val classManifest: ClassManifest[(K, V)] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]] + override val classTag: ClassTag[(K, V)] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K, V]]] } object JavaPairDStream { - implicit def fromPairDStream[K: ClassManifest, V: ClassManifest](dstream: DStream[(K, V)]) + implicit def fromPairDStream[K: ClassTag, V: ClassTag](dstream: DStream[(K, V)]) :JavaPairDStream[K, V] = new JavaPairDStream[K, V](dstream) def fromJavaDStream[K, V](dstream: JavaDStream[(K, V)]): JavaPairDStream[K, V] = { - implicit val cmk: ClassManifest[K] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]] - implicit val cmv: ClassManifest[V] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]] + implicit val cmk: ClassTag[K] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val cmv: ClassTag[V] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] new JavaPairDStream[K, V](dstream.dstream) } - def scalaToJavaLong[K: ClassManifest](dstream: JavaPairDStream[K, Long]) + def scalaToJavaLong[K: ClassTag](dstream: JavaPairDStream[K, Long]) : JavaPairDStream[K, JLong] = { StreamingContext.toPairDStreamFunctions(dstream.dstream).mapValues(new JLong(_)) } diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 3d149a742c..00e5aa0603 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -17,6 +17,7 @@ import akka.actor.SupervisorStrategy import akka.zeromq.Subscribe import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import java.lang.{Long => JLong, Integer => JInt} import java.io.InputStream @@ -126,8 +127,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { groupId: String, topics: JMap[String, JInt]) : JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cmt: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.kafkaStream[T](zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) } @@ -146,8 +147,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { topics: JMap[String, JInt], initialOffsets: JMap[KafkaPartitionKey, JLong]) : JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cmt: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.kafkaStream[T]( zkQuorum, groupId, @@ -172,8 +173,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { initialOffsets: JMap[KafkaPartitionKey, JLong], storageLevel: StorageLevel) : JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cmt: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.kafkaStream[T]( zkQuorum, groupId, @@ -224,8 +225,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { storageLevel: StorageLevel) : JavaDStream[T] = { def fn = (x: InputStream) => converter.apply(x).toIterator - implicit val cmt: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cmt: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.socketStream(hostname, port, fn, storageLevel) } @@ -253,8 +254,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { hostname: String, port: Int, storageLevel: StorageLevel): JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cmt: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port, storageLevel)) } @@ -268,8 +269,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @tparam T Type of the objects in the received blocks */ def rawSocketStream[T](hostname: String, port: Int): JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cmt: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port)) } @@ -283,12 +284,12 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @tparam F Input format for reading HDFS file */ def fileStream[K, V, F <: NewInputFormat[K, V]](directory: String): JavaPairDStream[K, V] = { - implicit val cmk: ClassManifest[K] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]] - implicit val cmv: ClassManifest[V] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]] - implicit val cmf: ClassManifest[F] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[F]] + implicit val cmk: ClassTag[K] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val cmv: ClassTag[V] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + implicit val cmf: ClassTag[F] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[F]] ssc.fileStream[K, V, F](directory); } @@ -372,8 +373,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { storageLevel: StorageLevel, supervisorStrategy: SupervisorStrategy ): JavaDStream[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.actorStream[T](props, name, storageLevel, supervisorStrategy) } @@ -393,8 +394,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { name: String, storageLevel: StorageLevel ): JavaDStream[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.actorStream[T](props, name, storageLevel) } @@ -412,8 +413,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { props: Props, name: String ): JavaDStream[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.actorStream[T](props, name) } @@ -434,8 +435,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { storageLevel: StorageLevel, supervisorStrategy: SupervisorStrategy ): JavaDStream[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.zeroMQStream[T](publisherUrl, subscribe, bytesToObjects, storageLevel, supervisorStrategy) } @@ -455,8 +456,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], storageLevel: StorageLevel ): JavaDStream[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel) } @@ -475,8 +476,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { subscribe: Subscribe, bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]] ): JavaDStream[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator ssc.zeroMQStream[T](publisherUrl, subscribe, fn) } @@ -497,8 +498,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @tparam T Type of objects in the RDD */ def queueStream[T](queue: java.util.Queue[JavaRDD[T]]): JavaDStream[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]] sQueue.enqueue(queue.map(_.rdd).toSeq: _*) ssc.queueStream(sQueue) @@ -514,8 +515,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @tparam T Type of objects in the RDD */ def queueStream[T](queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean): JavaDStream[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]] sQueue.enqueue(queue.map(_.rdd).toSeq: _*) ssc.queueStream(sQueue, oneAtATime) @@ -535,8 +536,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean, defaultRDD: JavaRDD[T]): JavaDStream[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[spark.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/spark/streaming/dstream/CoGroupedDStream.scala index 4ef4bb7de1..3e974b2731 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala @@ -4,8 +4,10 @@ import spark.{RDD, Partitioner} import spark.rdd.CoGroupedRDD import spark.streaming.{Time, DStream, Duration} +import scala.reflect.ClassTag + private[streaming] -class CoGroupedDStream[K : ClassManifest]( +class CoGroupedDStream[K : ClassTag]( parents: Seq[DStream[(K, _)]], partitioner: Partitioner ) extends DStream[(K, Seq[Seq[_]])](parents.head.ssc) { diff --git a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala index 41c3af4694..780f361869 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala @@ -3,10 +3,12 @@ package spark.streaming.dstream import spark.RDD import spark.streaming.{Time, StreamingContext} +import scala.reflect.ClassTag + /** * An input stream that always returns the same RDD on each timestep. Useful for testing. */ -class ConstantInputDStream[T: ClassManifest](ssc_ : StreamingContext, rdd: RDD[T]) +class ConstantInputDStream[T: ClassTag](ssc_ : StreamingContext, rdd: RDD[T]) extends InputDStream[T](ssc_) { override def start() {} @@ -16,4 +18,4 @@ class ConstantInputDStream[T: ClassManifest](ssc_ : StreamingContext, rdd: RDD[T override def compute(validTime: Time): Option[RDD[T]] = { Some(rdd) } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala index 41b9bd9461..78e62ca1d2 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala @@ -9,14 +9,16 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import scala.collection.mutable.{HashSet, HashMap} +import scala.reflect.ClassTag + import java.io.{ObjectInputStream, IOException} private[streaming] -class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K,V] : ClassManifest]( +class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : ClassTag]( @transient ssc_ : StreamingContext, directory: String, filter: Path => Boolean = FileInputDStream.defaultFilter, - newFilesOnly: Boolean = true) + newFilesOnly: Boolean = true) extends InputDStream[(K, V)](ssc_) { protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData @@ -37,7 +39,7 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K } logDebug("LastModTime initialized to " + lastModTime + ", new files only = " + newFilesOnly) } - + override def stop() { } /** @@ -83,7 +85,7 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K latestModTimeFiles += path.toString logDebug("Accepted " + path) return true - } + } } } logDebug("Finding new files at time " + validTime + " for last mod time = " + lastModTime) @@ -178,5 +180,3 @@ private[streaming] object FileInputDStream { def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".") } - - diff --git a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala index e993164f99..3f6054994e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala @@ -3,8 +3,10 @@ package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} import spark.RDD +import scala.reflect.ClassTag + private[streaming] -class FilteredDStream[T: ClassManifest]( +class FilteredDStream[T: ClassTag]( parent: DStream[T], filterFunc: T => Boolean ) extends DStream[T](parent.ssc) { @@ -17,5 +19,3 @@ class FilteredDStream[T: ClassManifest]( parent.getOrCompute(validTime).map(_.filter(filterFunc)) } } - - diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala index cabd34f5f2..66d28b9841 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala @@ -4,8 +4,10 @@ import spark.streaming.{Duration, DStream, Time} import spark.RDD import spark.SparkContext._ +import scala.reflect.ClassTag + private[streaming] -class FlatMapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest]( +class FlatMapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag]( parent: DStream[(K, V)], flatMapValueFunc: V => TraversableOnce[U] ) extends DStream[(K, U)](parent.ssc) { diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala index a69af60589..b946d9c689 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala @@ -3,8 +3,10 @@ package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} import spark.RDD +import scala.reflect.ClassTag + private[streaming] -class FlatMappedDStream[T: ClassManifest, U: ClassManifest]( +class FlatMappedDStream[T: ClassTag, U: ClassTag]( parent: DStream[T], flatMapFunc: T => Traversable[U] ) extends DStream[U](parent.ssc) { @@ -17,4 +19,3 @@ class FlatMappedDStream[T: ClassManifest, U: ClassManifest]( parent.getOrCompute(validTime).map(_.flatMap(flatMapFunc)) } } - diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala index c9644b3a83..640033c86c 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala @@ -12,13 +12,14 @@ import org.apache.avro.ipc.specific.SpecificResponder import org.apache.avro.ipc.NettyServer import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import java.net.InetSocketAddress import java.io.{ObjectInput, ObjectOutput, Externalizable} import java.nio.ByteBuffer private[streaming] -class FlumeInputDStream[T: ClassManifest]( +class FlumeInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, host: String, port: Int, diff --git a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala index ee69ea5177..2494273ce6 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala @@ -2,9 +2,10 @@ package spark.streaming.dstream import spark.RDD import spark.streaming.{Duration, DStream, Job, Time} +import scala.reflect.ClassTag private[streaming] -class ForEachDStream[T: ClassManifest] ( +class ForEachDStream[T: ClassTag] ( parent: DStream[T], foreachFunc: (RDD[T], Time) => Unit ) extends DStream[Unit](parent.ssc) { diff --git a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala index b589cbd4d5..658c49b61b 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala @@ -3,8 +3,10 @@ package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} import spark.RDD +import scala.reflect.ClassTag + private[streaming] -class GlommedDStream[T: ClassManifest](parent: DStream[T]) +class GlommedDStream[T: ClassTag](parent: DStream[T]) extends DStream[Array[T]](parent.ssc) { override def dependencies = List(parent) diff --git a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala index 3c5d43a609..f9451281d6 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala @@ -2,6 +2,8 @@ package spark.streaming.dstream import spark.streaming.{Time, Duration, StreamingContext, DStream} +import scala.reflect.ClassTag + /** * This is the abstract base class for all input streams. This class provides to methods * start() and stop() which called by the scheduler to start and stop receiving data/ @@ -13,7 +15,7 @@ import spark.streaming.{Time, Duration, StreamingContext, DStream} * that requires running a receiver on the worker nodes, use NetworkInputDStream * as the parent class. */ -abstract class InputDStream[T: ClassManifest] (@transient ssc_ : StreamingContext) +abstract class InputDStream[T: ClassTag] (@transient ssc_ : StreamingContext) extends DStream[T](ssc_) { var lastValidTime: Time = null diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index ddd9becf32..e093edb05b 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -16,14 +16,14 @@ import kafka.utils.ZkUtils._ import scala.collection.Map import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ - +import scala.reflect.ClassTag // 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 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 @@ -33,7 +33,7 @@ case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, part * @param storageLevel RDD storage level. */ private[streaming] -class KafkaInputDStream[T: ClassManifest]( +class KafkaInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, zkQuorum: String, groupId: String, @@ -51,7 +51,7 @@ class KafkaInputDStream[T: ClassManifest]( private[streaming] class KafkaReceiver(zkQuorum: String, groupId: String, - topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], + topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], storageLevel: StorageLevel) extends NetworkReceiver[Any] { // Timeout for establishing a connection to Zookeper in ms. diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala index 848afecfad..382739932b 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala @@ -3,8 +3,10 @@ package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} import spark.RDD +import scala.reflect.ClassTag + private[streaming] -class MapPartitionedDStream[T: ClassManifest, U: ClassManifest]( +class MapPartitionedDStream[T: ClassTag, U: ClassTag]( parent: DStream[T], mapPartFunc: Iterator[T] => Iterator[U], preservePartitioning: Boolean @@ -18,4 +20,3 @@ class MapPartitionedDStream[T: ClassManifest, U: ClassManifest]( parent.getOrCompute(validTime).map(_.mapPartitions[U](mapPartFunc, preservePartitioning)) } } - diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala index 6055aa6a05..9c508eb854 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala @@ -4,8 +4,10 @@ import spark.streaming.{Duration, DStream, Time} import spark.RDD import spark.SparkContext._ +import scala.reflect.ClassTag + private[streaming] -class MapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest]( +class MapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag]( parent: DStream[(K, V)], mapValueFunc: V => U ) extends DStream[(K, U)](parent.ssc) { @@ -18,4 +20,3 @@ class MapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest]( parent.getOrCompute(validTime).map(_.mapValues[U](mapValueFunc)) } } - diff --git a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala index 20818a0cab..c7cbf6cc46 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala @@ -3,8 +3,10 @@ package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} import spark.RDD +import scala.reflect.ClassTag + private[streaming] -class MappedDStream[T: ClassManifest, U: ClassManifest] ( +class MappedDStream[T: ClassTag, U: ClassTag] ( parent: DStream[T], mapFunc: T => U ) extends DStream[U](parent.ssc) { @@ -17,4 +19,3 @@ class MappedDStream[T: ClassManifest, U: ClassManifest] ( parent.getOrCompute(validTime).map(_.map[U](mapFunc)) } } - diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index 5347374730..52b9968f6e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -8,6 +8,7 @@ import spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ +import scala.reflect.ClassTag import java.nio.ByteBuffer @@ -28,7 +29,7 @@ import java.util.concurrent.ArrayBlockingQueue * @param ssc_ Streaming context that will execute this input stream * @tparam T Class type of the object of this stream */ -abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : StreamingContext) +abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) extends InputDStream[T](ssc_) { // This is an unique identifier that is used to match the network receiver with the @@ -70,7 +71,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. */ -abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Logging { +abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging { initLogging() diff --git a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala index 3c2a81947b..70deec6ec9 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala @@ -2,8 +2,10 @@ package spark.streaming.dstream import spark.streaming.StreamingContext +import scala.reflect.ClassTag + private[streaming] -class PluggableInputDStream[T: ClassManifest]( +class PluggableInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, receiver: NetworkReceiver[T]) extends NetworkInputDStream[T](ssc_) { diff --git a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala index da224ad6f7..8b03a49fd6 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala @@ -7,18 +7,20 @@ import scala.collection.mutable.Queue import scala.collection.mutable.ArrayBuffer import spark.streaming.{Time, StreamingContext} +import scala.reflect.ClassTag + private[streaming] -class QueueInputDStream[T: ClassManifest]( +class QueueInputDStream[T: ClassTag]( @transient ssc: StreamingContext, val queue: Queue[RDD[T]], oneAtATime: Boolean, defaultRDD: RDD[T] ) extends InputDStream[T](ssc) { - + override def start() { } - + override def stop() { } - + override def compute(validTime: Time): Option[RDD[T]] = { val buffer = new ArrayBuffer[RDD[T]]() if (oneAtATime && queue.size > 0) { @@ -38,5 +40,5 @@ class QueueInputDStream[T: ClassManifest]( None } } - + } diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala index 1b2fa56779..7d20ac8e95 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala @@ -4,6 +4,8 @@ import spark.Logging import spark.storage.StorageLevel import spark.streaming.StreamingContext +import scala.reflect.ClassTag + import java.net.InetSocketAddress import java.nio.ByteBuffer import java.nio.channels.{ReadableByteChannel, SocketChannel} @@ -18,7 +20,7 @@ import java.util.concurrent.ArrayBlockingQueue * in the format that the system is configured with. */ private[streaming] -class RawInputDStream[T: ClassManifest]( +class RawInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, host: String, port: Int, diff --git a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala index 343b6915e7..c9ada5b306 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -8,11 +8,13 @@ import spark.Partitioner import spark.SparkContext._ import spark.storage.StorageLevel -import scala.collection.mutable.ArrayBuffer import spark.streaming.{Duration, Interval, Time, DStream} +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + private[streaming] -class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest]( +class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( parent: DStream[(K, V)], reduceFunc: (V, V) => V, invReduceFunc: (V, V) => V, @@ -32,7 +34,7 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest]( "must be multiple of the slide duration of parent DStream (" + parent.slideDuration + ")" ) - // Reduce each batch of data using reduceByKey which will be further reduced by window + // Reduce each batch of data using reduceByKey which will be further reduced by window // by ReducedWindowedDStream val reducedStream = parent.reduceByKey(reduceFunc, partitioner) @@ -153,5 +155,3 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest]( } } } - - diff --git a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala index 1f9548bfb8..930e6c2044 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala @@ -4,8 +4,10 @@ import spark.{RDD, Partitioner} import spark.SparkContext._ import spark.streaming.{Duration, DStream, Time} +import scala.reflect.ClassTag + private[streaming] -class ShuffledDStream[K: ClassManifest, V: ClassManifest, C: ClassManifest]( +class ShuffledDStream[K: ClassTag, V: ClassTag, C: ClassTag]( parent: DStream[(K,V)], createCombiner: V => C, mergeValue: (C, V) => C, diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala index 1408af0afa..c9e701e07f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala @@ -4,11 +4,13 @@ import spark.streaming.StreamingContext import spark.storage.StorageLevel import spark.util.NextIterator +import scala.reflect.ClassTag + import java.io._ import java.net.Socket private[streaming] -class SocketInputDStream[T: ClassManifest]( +class SocketInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, host: String, port: Int, @@ -22,7 +24,7 @@ class SocketInputDStream[T: ClassManifest]( } private[streaming] -class SocketReceiver[T: ClassManifest]( +class SocketReceiver[T: ClassTag]( host: String, port: Int, bytesToObjects: InputStream => Iterator[T], diff --git a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala index db62955036..f6fb2ca941 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala @@ -6,8 +6,10 @@ import spark.SparkContext._ import spark.storage.StorageLevel import spark.streaming.{Duration, Time, DStream} +import scala.reflect.ClassTag + private[streaming] -class StateDStream[K: ClassManifest, V: ClassManifest, S: ClassManifest]( +class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( parent: DStream[(K, V)], updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)], partitioner: Partitioner, diff --git a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala index 99660d9dee..23ab87d82f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala @@ -3,8 +3,10 @@ package spark.streaming.dstream import spark.RDD import spark.streaming.{Duration, DStream, Time} +import scala.reflect.ClassTag + private[streaming] -class TransformedDStream[T: ClassManifest, U: ClassManifest] ( +class TransformedDStream[T: ClassTag, U: ClassTag] ( parent: DStream[T], transformFunc: (RDD[T], Time) => RDD[U] ) extends DStream[U](parent.ssc) { diff --git a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala index 00bad5da34..3a5bf35249 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala @@ -2,11 +2,13 @@ package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} import spark.RDD -import collection.mutable.ArrayBuffer import spark.rdd.UnionRDD +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + private[streaming] -class UnionDStream[T: ClassManifest](parents: Array[DStream[T]]) +class UnionDStream[T: ClassTag](parents: Array[DStream[T]]) extends DStream[T](parents.head.ssc) { if (parents.length == 0) { diff --git a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala index cbf0c88108..2382210b0e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala @@ -5,8 +5,10 @@ import spark.rdd.UnionRDD import spark.storage.StorageLevel import spark.streaming.{Duration, Interval, Time, DStream} +import scala.reflect.ClassTag + private[streaming] -class WindowedDStream[T: ClassManifest]( +class WindowedDStream[T: ClassTag]( parent: DStream[T], _windowDuration: Duration, _slideDuration: Duration) @@ -35,6 +37,3 @@ class WindowedDStream[T: ClassManifest]( Some(new UnionRDD(ssc.sc, parent.slice(currentWindow))) } } - - - diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala index 6c9e373de3..42e885af5c 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala @@ -4,14 +4,16 @@ import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy } import akka.actor.{ actorRef2Scala, ActorRef } import akka.actor.{ PossiblyHarmful, OneForOneStrategy } import akka.actor.SupervisorStrategy._ + import scala.concurrent.duration._ +import scala.reflect.ClassTag import spark.storage.StorageLevel import spark.streaming.dstream.NetworkReceiver import java.util.concurrent.atomic.AtomicInteger -/** A helper with set of defaults for supervisor strategy **/ +/** A helper with set of defaults for supervisor strategy */ object ReceiverSupervisorStrategy { val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = @@ -43,11 +45,11 @@ object ReceiverSupervisorStrategy { * */ trait Receiver { self: Actor ⇒ - def pushBlock[T: ClassManifest](iter: Iterator[T]) { + def pushBlock[T: ClassTag](iter: Iterator[T]) { context.parent ! Data(iter) } - def pushBlock[T: ClassManifest](data: T) { + def pushBlock[T: ClassTag](data: T) { context.parent ! Data(data) } @@ -61,8 +63,8 @@ case class Statistics(numberOfMsgs: Int, numberOfHiccups: Int, otherInfo: String) -/** Case class to receive data sent by child actors **/ -private[streaming] case class Data[T: ClassManifest](data: T) +/** Case class to receive data sent by child actors */ +private[streaming] case class Data[T: ClassTag](data: T) /** * Provides Actors as receivers for receiving stream. @@ -85,7 +87,7 @@ private[streaming] case class Data[T: ClassManifest](data: T) * * */ -private[streaming] class ActorReceiver[T: ClassManifest]( +private[streaming] class ActorReceiver[T: ClassTag]( props: Props, name: String, storageLevel: StorageLevel, diff --git a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala index e7608f08ae..45e2596707 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala @@ -5,10 +5,12 @@ import akka.zeromq._ import spark.Logging +import scala.reflect.ClassTag + /** * A receiver to subscribe to ZeroMQ stream. */ -private[streaming] class ZeroMQReceiver[T: ClassManifest](publisherUrl: String, +private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, subscribe: Subscribe, bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T]) extends Actor with Receiver with Logging { diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala index f673e5be15..3db1eaa834 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala @@ -7,6 +7,7 @@ import StreamingContext._ import scala.util.Random import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import scala.reflect.ClassTag import java.io.{File, ObjectInputStream, IOException} import java.util.UUID @@ -102,7 +103,7 @@ object MasterFailureTest extends Logging { * Tests stream operation with multiple master failures, and verifies whether the * final set of output values is as expected or not. */ - def testOperation[T: ClassManifest]( + def testOperation[T: ClassTag]( directory: String, batchDuration: Duration, input: Seq[String], @@ -140,7 +141,7 @@ object MasterFailureTest extends Logging { * and batch duration. Returns the streaming context and the directory to which * files should be written for testing. */ - private def setupStreams[T: ClassManifest]( + private def setupStreams[T: ClassTag]( directory: String, batchDuration: Duration, operation: DStream[String] => DStream[T] @@ -173,7 +174,7 @@ object MasterFailureTest extends Logging { * Repeatedly starts and kills the streaming context until timed out or * the last expected output is generated. Finally, return */ - private def runStreams[T: ClassManifest]( + private def runStreams[T: ClassTag]( ssc_ : StreamingContext, lastExpectedOutput: T, maxTimeToRun: Long @@ -254,7 +255,7 @@ object MasterFailureTest extends Logging { * duplicate batch outputs of values from the `output`. As a result, the * expected output should not have consecutive batches with the same values as output. */ - private def verifyOutput[T: ClassManifest](output: Seq[T], expectedOutput: Seq[T]) { + private def verifyOutput[T: ClassTag](output: Seq[T], expectedOutput: Seq[T]) { // Verify whether expected outputs do not consecutive batches with same output for (i <- 0 until expectedOutput.size - 1) { assert(expectedOutput(i) != expectedOutput(i+1), @@ -285,7 +286,7 @@ object MasterFailureTest extends Logging { * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint. */ private[streaming] -class TestOutputStream[T: ClassManifest]( +class TestOutputStream[T: ClassTag]( parent: DStream[T], val output: ArrayBuffer[Seq[T]] = new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]] ) extends ForEachDStream[T]( @@ -359,22 +360,22 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) val hadoopFile = new Path(testDir, (i+1).toString) FileUtils.writeStringToFile(localFile, input(i).toString + "\n") var tries = 0 - var done = false + var done = false while (!done && tries < maxTries) { tries += 1 try { fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile) - done = true - } catch { - case ioe: IOException => { - fs = testDir.getFileSystem(new Configuration()) + done = true + } catch { + case ioe: IOException => { + fs = testDir.getFileSystem(new Configuration()) logWarning("Attempt " + tries + " at generating file " + hadoopFile + " failed.", ioe) - } - } } - if (!done) + } + } + if (!done) logError("Could not generate file " + hadoopFile) - else + else logInfo("Generated file " + hadoopFile + " at " + System.currentTimeMillis) Thread.sleep(interval) localFile.delete() @@ -388,5 +389,3 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) } } } - - diff --git a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala index 64a7e7cbf9..8a7c48bde6 100644 --- a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala @@ -1,6 +1,8 @@ package spark.streaming -import collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import scala.reflect.ClassTag + import java.util.{List => JList} import spark.streaming.api.java.{JavaPairDStream, JavaDStreamLike, JavaDStream, JavaStreamingContext} import spark.streaming._ @@ -13,15 +15,15 @@ trait JavaTestBase extends TestSuiteBase { /** * Create a [[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]( ssc: JavaStreamingContext, data: JList[JList[T]], numPartitions: Int) = { val seqData = data.map(Seq(_:_*)) - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val dstream = new TestInputStream[T](ssc.ssc, seqData, numPartitions) ssc.ssc.registerInputStream(dstream) new JavaDStream[T](dstream) @@ -30,12 +32,12 @@ trait JavaTestBase extends TestSuiteBase { /** * Attach a provided stream to it's associated StreamingContext as a * [[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]) = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val ostream = new TestOutputStream(dstream.dstream, new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]]) dstream.dstream.ssc.registerOutputStream(ostream) @@ -48,8 +50,8 @@ trait JavaTestBase extends TestSuiteBase { */ def runStreams[V]( ssc: JavaStreamingContext, numBatches: Int, numExpectedOutput: Int): JList[JList[V]] = { - implicit val cm: ClassManifest[V] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]] + implicit val cm: ClassTag[V] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] val res = runStreams[V](ssc.ssc, numBatches, numExpectedOutput) val out = new ArrayList[JList[V]]() res.map(entry => out.append(new ArrayList[V](entry))) @@ -64,4 +66,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/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala index f9285b19e2..143a26d911 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala @@ -3,6 +3,7 @@ package spark.streaming import java.io.File import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag import org.apache.commons.io.FileUtils import org.scalatest.BeforeAndAfter @@ -297,7 +298,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { * NOTE: This takes into consideration that the last batch processed before * master failure will be re-processed after restart/recovery. */ - def testCheckpointedOperation[U: ClassManifest, V: ClassManifest]( + def testCheckpointedOperation[U: ClassTag, V: ClassTag]( input: Seq[Seq[U]], operation: DStream[U] => DStream[V], expectedOutput: Seq[Seq[V]], @@ -340,7 +341,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { * Advances the manual clock on the streaming scheduler by given number of batches. * It also waits for the expected amount of time for each batch. */ - def advanceTimeWithRealDelay[V: ClassManifest](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = { + def advanceTimeWithRealDelay[V: ClassTag](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] logInfo("Manual clock before advancing = " + clock.time) for (i <- 1 to numBatches.toInt) { diff --git a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala index ad6aa79d10..dc280b09c9 100644 --- a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala @@ -5,8 +5,9 @@ import spark.streaming.util.ManualClock import spark.{RDD, Logging} -import collection.mutable.ArrayBuffer -import collection.mutable.SynchronizedBuffer +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.SynchronizedBuffer +import scala.reflect.ClassTag import java.io.{ObjectInputStream, IOException} @@ -17,7 +18,7 @@ import org.scalatest.{BeforeAndAfter, FunSuite} * replayable, reliable message queue like Kafka. It requires a sequence as input, and * returns the i_th element at the i_th batch unde manual clock. */ -class TestInputStream[T: ClassManifest](ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int) +class TestInputStream[T: ClassTag](ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int) extends InputDStream[T](ssc_) { def start() {} @@ -43,7 +44,7 @@ class TestInputStream[T: ClassManifest](ssc_ : StreamingContext, input: Seq[Seq[ * This is a output stream just for the testsuites. All the output is collected into a * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint. */ -class TestOutputStream[T: ClassManifest](parent: DStream[T], val output: ArrayBuffer[Seq[T]]) +class TestOutputStream[T: ClassTag](parent: DStream[T], val output: ArrayBuffer[Seq[T]]) extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => { val collected = rdd.collect() output += collected @@ -88,7 +89,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { * Set up required DStreams to test the DStream operation using the two sequences * of input collections. */ - def setupStreams[U: ClassManifest, V: ClassManifest]( + def setupStreams[U: ClassTag, V: ClassTag]( input: Seq[Seq[U]], operation: DStream[U] => DStream[V] ): StreamingContext = { @@ -112,7 +113,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { * Set up required DStreams to test the binary operation using the sequence * of input collections. */ - def setupStreams[U: ClassManifest, V: ClassManifest, W: ClassManifest]( + def setupStreams[U: ClassTag, V: ClassTag, W: ClassTag]( input1: Seq[Seq[U]], input2: Seq[Seq[V]], operation: (DStream[U], DStream[V]) => DStream[W] @@ -140,7 +141,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { * returns the collected output. It will wait until `numExpectedOutput` number of * output data has been collected or timeout (set by `maxWaitTimeMillis`) is reached. */ - def runStreams[V: ClassManifest]( + def runStreams[V: ClassTag]( ssc: StreamingContext, numBatches: Int, numExpectedOutput: Int @@ -196,7 +197,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { * is same as the expected output values, by comparing the output * collections either as lists (order matters) or sets (order does not matter) */ - def verifyOutput[V: ClassManifest]( + def verifyOutput[V: ClassTag]( output: Seq[Seq[V]], expectedOutput: Seq[Seq[V]], useSet: Boolean @@ -226,7 +227,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { * Test unary DStream operation with a list of inputs, with number of * batches to run same as the number of expected output values */ - def testOperation[U: ClassManifest, V: ClassManifest]( + def testOperation[U: ClassTag, V: ClassTag]( input: Seq[Seq[U]], operation: DStream[U] => DStream[V], expectedOutput: Seq[Seq[V]], @@ -244,7 +245,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { * @param useSet Compare the output values with the expected output values * as sets (order matters) or as lists (order does not matter) */ - def testOperation[U: ClassManifest, V: ClassManifest]( + def testOperation[U: ClassTag, V: ClassTag]( input: Seq[Seq[U]], operation: DStream[U] => DStream[V], expectedOutput: Seq[Seq[V]], @@ -261,7 +262,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { * Test binary DStream operation with two lists of inputs, with number of * batches to run same as the number of expected output values */ - def testOperation[U: ClassManifest, V: ClassManifest, W: ClassManifest]( + def testOperation[U: ClassTag, V: ClassTag, W: ClassTag]( input1: Seq[Seq[U]], input2: Seq[Seq[V]], operation: (DStream[U], DStream[V]) => DStream[W], @@ -281,7 +282,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { * @param useSet Compare the output values with the expected output values * as sets (order matters) or as lists (order does not matter) */ - def testOperation[U: ClassManifest, V: ClassManifest, W: ClassManifest]( + def testOperation[U: ClassTag, V: ClassTag, W: ClassTag]( input1: Seq[Seq[U]], input2: Seq[Seq[V]], operation: (DStream[U], DStream[V]) => DStream[W], -- cgit v1.2.3 From d3518f57cd833f6297860c6344e685b9bf0210f5 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 29 Apr 2013 18:14:25 +0530 Subject: Fixed warning: erasure -> runtimeClass --- core/src/main/scala/spark/PairRDDFunctions.scala | 8 ++++---- core/src/main/scala/spark/RDD.scala | 4 ++-- .../main/scala/spark/SequenceFileRDDFunctions.scala | 4 ++-- core/src/main/scala/spark/SparkContext.scala | 18 +++++++++--------- .../scala/spark/streaming/PairDStreamFunctions.scala | 8 ++++---- 5 files changed, 21 insertions(+), 21 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 0453e98e7d..2052d05788 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -512,7 +512,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag]( * supporting the key and value types K and V in this RDD. */ def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { - saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) + saveAsHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -520,7 +520,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag]( * (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD. */ def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { - saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) + saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -651,9 +651,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag]( */ def values: RDD[V] = self.map(_._2) - private[spark] def getKeyClass() = implicitly[ClassTag[K]].erasure + private[spark] def getKeyClass() = implicitly[ClassTag[K]].runtimeClass - private[spark] def getValueClass() = implicitly[ClassTag[V]].erasure + private[spark] def getValueClass() = implicitly[ClassTag[V]].runtimeClass } /** diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index e6e0997a59..6ee075315a 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -608,7 +608,7 @@ abstract class RDD[T: ClassTag]( * combine step happens locally on the master, equivalent to running a single reduce task. */ def countByValue(): Map[T, Long] = { - if (elementClassTag.erasure.isArray) { + if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValue() does not support arrays") } // TODO: This should perhaps be distributed by default. @@ -639,7 +639,7 @@ abstract class RDD[T: ClassTag]( timeout: Long, confidence: Double = 0.95 ): PartialResult[Map[T, BoundedDouble]] = { - if (elementClassTag.erasure.isArray) { + if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } val countPartition: (TaskContext, Iterator[T]) => OLMap[T] = { (ctx, iter) => diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala index 900d73bf42..883a0152bb 100644 --- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala @@ -40,8 +40,8 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag private def getWritableClass[T <% Writable: ClassTag](): Class[_ <: Writable] = { val c = { - if (classOf[Writable].isAssignableFrom(classTag[T].erasure)) { - classTag[T].erasure + if (classOf[Writable].isAssignableFrom(classTag[T].runtimeClass)) { + classTag[T].runtimeClass } else { // We get the type of the Writable class by looking at the apply method which converts // from T to Writable. Since we have two apply methods we filter out the one which diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 6bd87bf3ec..7272a592a5 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -277,9 +277,9 @@ class SparkContext( (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]) : RDD[(K, V)] = { hadoopFile(path, - fm.erasure.asInstanceOf[Class[F]], - km.erasure.asInstanceOf[Class[K]], - vm.erasure.asInstanceOf[Class[V]], + fm.runtimeClass.asInstanceOf[Class[F]], + km.runtimeClass.asInstanceOf[Class[K]], + vm.runtimeClass.asInstanceOf[Class[V]], minSplits) } @@ -300,9 +300,9 @@ class SparkContext( (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = { newAPIHadoopFile( path, - fm.erasure.asInstanceOf[Class[F]], - km.erasure.asInstanceOf[Class[K]], - vm.erasure.asInstanceOf[Class[V]]) + fm.runtimeClass.asInstanceOf[Class[F]], + km.runtimeClass.asInstanceOf[Class[K]], + vm.runtimeClass.asInstanceOf[Class[V]]) } /** @@ -781,13 +781,13 @@ object SparkContext { private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T]): ArrayWritable = { def anyToWritable[U <% Writable](u: U): Writable = u - new ArrayWritable(classTag[T].erasure.asInstanceOf[Class[Writable]], + new ArrayWritable(classTag[T].runtimeClass.asInstanceOf[Class[Writable]], arr.map(x => anyToWritable(x)).toArray) } // Helper objects for converting common types to Writable private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) = { - val wClass = classTag[W].erasure.asInstanceOf[Class[W]] + val wClass = classTag[W].runtimeClass.asInstanceOf[Class[W]] new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W])) } @@ -806,7 +806,7 @@ object SparkContext { implicit def stringWritableConverter() = simpleWritableConverter[String, Text](_.toString) implicit def writableWritableConverter[T <: Writable]() = - new WritableConverter[T](_.erasure.asInstanceOf[Class[T]], _.asInstanceOf[T]) + new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T]) /** * Find the JAR from which a given class was loaded, to make it easy for users to pass diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index 962ba6619d..aa1a8b6ba2 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -457,7 +457,7 @@ extends Serializable { prefix: String, suffix: String )(implicit fm: ClassTag[F]) { - saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) + saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -487,7 +487,7 @@ extends Serializable { prefix: String, suffix: String )(implicit fm: ClassTag[F]) { - saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) + saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -509,7 +509,7 @@ extends Serializable { self.foreach(saveFunc) } - private def getKeyClass() = implicitly[ClassTag[K]].erasure + private def getKeyClass() = implicitly[ClassTag[K]].runtimeClass - private def getValueClass() = implicitly[ClassTag[V]].erasure + private def getValueClass() = implicitly[ClassTag[V]].runtimeClass } -- cgit v1.2.3 From 24bbf318b3e8e657f911204b84f2d032fb1ff6e2 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 29 Apr 2013 19:56:28 +0530 Subject: Fixied other warnings --- core/src/main/scala/spark/TaskState.scala | 3 +-- core/src/main/scala/spark/api/python/PythonRDD.scala | 4 ++-- core/src/main/scala/spark/deploy/ExecutorState.scala | 3 +-- core/src/main/scala/spark/deploy/master/ApplicationState.scala | 3 +-- core/src/main/scala/spark/deploy/master/WorkerState.scala | 2 +- core/src/main/scala/spark/util/AkkaUtils.scala | 4 ++-- repl/src/main/scala/spark/repl/SparkIMain.scala | 2 +- streaming/src/main/scala/spark/streaming/DStream.scala | 4 +--- 8 files changed, 10 insertions(+), 15 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/spark/TaskState.scala b/core/src/main/scala/spark/TaskState.scala index 78eb33a628..44893ef089 100644 --- a/core/src/main/scala/spark/TaskState.scala +++ b/core/src/main/scala/spark/TaskState.scala @@ -2,8 +2,7 @@ package spark import org.apache.mesos.Protos.{TaskState => MesosTaskState} -private[spark] object TaskState - extends Enumeration("LAUNCHING", "RUNNING", "FINISHED", "FAILED", "KILLED", "LOST") { +private[spark] object TaskState extends Enumeration { val LAUNCHING, RUNNING, FINISHED, FAILED, KILLED, LOST = Value diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 82959a33eb..220047c360 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -134,7 +134,7 @@ private[spark] class PythonRDD[T: ClassTag]( } new Array[Byte](0) } - case e => throw e + case e : Throwable => throw e } } @@ -233,7 +233,7 @@ private[spark] object PythonRDD { } } catch { case eof: EOFException => {} - case e => throw e + case e : Throwable => throw e } JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } diff --git a/core/src/main/scala/spark/deploy/ExecutorState.scala b/core/src/main/scala/spark/deploy/ExecutorState.scala index 5dc0c54552..a487e11f2c 100644 --- a/core/src/main/scala/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/spark/deploy/ExecutorState.scala @@ -1,7 +1,6 @@ package spark.deploy -private[spark] object ExecutorState - extends Enumeration("LAUNCHING", "LOADING", "RUNNING", "KILLED", "FAILED", "LOST") { +private[spark] object ExecutorState extends Enumeration { val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST = Value diff --git a/core/src/main/scala/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/spark/deploy/master/ApplicationState.scala index 15016b388d..4f359711c5 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationState.scala @@ -1,7 +1,6 @@ package spark.deploy.master -private[spark] object ApplicationState - extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") { +private[spark] object ApplicationState extends Enumeration { type ApplicationState = Value diff --git a/core/src/main/scala/spark/deploy/master/WorkerState.scala b/core/src/main/scala/spark/deploy/master/WorkerState.scala index 0bf35014c8..1e347bee20 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerState.scala +++ b/core/src/main/scala/spark/deploy/master/WorkerState.scala @@ -1,6 +1,6 @@ package spark.deploy.master -private[spark] object WorkerState extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED") { +private[spark] object WorkerState extends Enumeration { type WorkerState = Value val ALIVE, DEAD, DECOMMISSIONED = Value diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 70338ec4dc..e16915c8e9 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -61,7 +61,7 @@ private[spark] object AkkaUtils { * Creates a Spray HTTP server bound to a given IP and port with a given Spray Route object to * handle requests. Returns the bound port or throws a SparkException on failure. */ - def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route) { + def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route) = { val ioWorker = IOExtension(actorSystem).ioBridge() val httpService = actorSystem.actorOf(Props(HttpServiceActor(route))) val server = actorSystem.actorOf( @@ -72,7 +72,7 @@ private[spark] object AkkaUtils { try { Await.result(future, timeout) match { case bound: HttpServer.Bound => - return server + server case other: Any => throw new SparkException("Failed to bind web UI to port " + port + ": " + other) } diff --git a/repl/src/main/scala/spark/repl/SparkIMain.scala b/repl/src/main/scala/spark/repl/SparkIMain.scala index 9894429ec1..b1977d6788 100644 --- a/repl/src/main/scala/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/spark/repl/SparkIMain.scala @@ -809,7 +809,7 @@ import spark.Logging // val readRoot = getRequiredModule(readPath) // the outermost wrapper // MATEI: Changed this to getClass because the root object is no longer a module (Scala singleton object) - val readRoot = definitions.getClass(newTypeName(readPath)) // the outermost wrapper + val readRoot = rootMirror.getClassByName(newTypeName(readPath)) // the outermost wrapper (accessPath split '.').foldLeft(readRoot: Symbol) { case (sym, "") => sym case (sym, name) => afterTyper(termMember(sym, name)) diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index c307c69611..6ad43dd9b5 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -466,9 +466,7 @@ abstract class DStream[T: ClassTag] ( * this DStream will be registered as an output stream and therefore materialized. */ def foreach(foreachFunc: (RDD[T], Time) => Unit) { - val newStream = new ForEachDStream(this, context.sparkContext.clean(foreachFunc)) - ssc.registerOutputStream(newStream) - newStream + ssc.registerOutputStream(new ForEachDStream(this, context.sparkContext.clean(foreachFunc))) } /** -- cgit v1.2.3 From e8a9d1cdf92ffd842130d6953d7024e7a9f67ca6 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 1 May 2013 11:35:02 +0530 Subject: Fixed Warning: expect -> expectResult --- core/src/test/scala/spark/SizeEstimatorSuite.scala | 72 +++++++++++----------- 1 file changed, 36 insertions(+), 36 deletions(-) (limited to 'core/src') diff --git a/core/src/test/scala/spark/SizeEstimatorSuite.scala b/core/src/test/scala/spark/SizeEstimatorSuite.scala index e235ef2f67..9f3aa6628d 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))) + expectResult(16)(SizeEstimator.estimate(new DummyClass1)) + expectResult(16)(SizeEstimator.estimate(new DummyClass2)) + expectResult(24)(SizeEstimator.estimate(new DummyClass3)) + expectResult(24)(SizeEstimator.estimate(new DummyClass4(null))) + expectResult(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3))) } // 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"))) + expectResult(40)(SizeEstimator.estimate(DummyString(""))) + expectResult(48)(SizeEstimator.estimate(DummyString("a"))) + expectResult(48)(SizeEstimator.estimate(DummyString("ab"))) + expectResult(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) } 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))) + expectResult(32)(SizeEstimator.estimate(new Array[Byte](10))) + expectResult(40)(SizeEstimator.estimate(new Array[Char](10))) + expectResult(40)(SizeEstimator.estimate(new Array[Short](10))) + expectResult(56)(SizeEstimator.estimate(new Array[Int](10))) + expectResult(96)(SizeEstimator.estimate(new Array[Long](10))) + expectResult(56)(SizeEstimator.estimate(new Array[Float](10))) + expectResult(96)(SizeEstimator.estimate(new Array[Double](10))) + expectResult(4016)(SizeEstimator.estimate(new Array[Int](1000))) + expectResult(8016)(SizeEstimator.estimate(new Array[Long](1000))) } 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))) + expectResult(56)(SizeEstimator.estimate(new Array[String](10))) + expectResult(56)(SizeEstimator.estimate(new Array[AnyRef](10))) // 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))) + expectResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass1))) + expectResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass2))) + expectResult(296)(SizeEstimator.estimate(Array.fill(10)(new DummyClass3))) + expectResult(56)(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2))) // Past size 100, our samples 100 elements, but we should still get the right size. - expect(28016)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3))) + expectResult(28016)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3))) // 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 + expectResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object + expectResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 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"))) + expectResult(40)(SizeEstimator.estimate(DummyString(""))) + expectResult(48)(SizeEstimator.estimate(DummyString("a"))) + expectResult(48)(SizeEstimator.estimate(DummyString("ab"))) + expectResult(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) 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"))) + expectResult(56)(SizeEstimator.estimate(DummyString(""))) + expectResult(64)(SizeEstimator.estimate(DummyString("a"))) + expectResult(64)(SizeEstimator.estimate(DummyString("ab"))) + expectResult(72)(SizeEstimator.estimate(DummyString("abcdefgh"))) resetOrClear("os.arch", arch) resetOrClear("spark.test.useCompressedOops", oops) -- cgit v1.2.3 From dbe2887da779f8688359de22e86a3815ae7cd396 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 1 May 2013 13:22:49 +0530 Subject: Fixed deprecated method warning --- core/src/main/scala/spark/util/TimeStampedHashMap.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/spark/util/TimeStampedHashMap.scala index 188f8910da..4afba0a4c3 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/spark/util/TimeStampedHashMap.scala @@ -58,7 +58,7 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() with spark.Logging { } override def filter(p: ((A, B)) => Boolean): Map[A, B] = { - JavaConversions.asScalaConcurrentMap(internalMap).map(kv => (kv._1, kv._2._1)).filter(p) + JavaConversions.mapAsScalaConcurrentMap(internalMap).map(kv => (kv._1, kv._2._1)).filter(p) } override def empty: Map[A, B] = new TimeStampedHashMap[A, B]() -- cgit v1.2.3 From 5b5b5aedbf01c84e49da884f361c1ca87f410628 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 8 Jun 2013 01:09:24 -0700 Subject: Fixed a few test issues due to Akka 2.1, as well as SBT memory. Unfortunately, in Akka 2.1, ActorSystem.awaitTermination hangs for remote actors, and Akka also leaves a non-daemon Netty thread even when run in daemon mode. Thus I had to comment out some of the calls to awaitTermination, and we still have one failing test. --- core/src/main/scala/spark/SparkEnv.scala | 3 +- .../scala/spark/deploy/LocalSparkCluster.scala | 8 +++- project/SparkBuild.scala | 44 +++++++++++++--------- sbt/sbt | 2 +- 4 files changed, 35 insertions(+), 22 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 7157fd2688..144ddea35f 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -42,7 +42,8 @@ class SparkEnv ( 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 - actorSystem.awaitTermination() + // UPDATE: In Akka 2.1.x, this hangs if there are remote actors, so we can't call it. + //actorSystem.awaitTermination() } } diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index 22319a96ca..6abaaeaa3f 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -43,9 +43,13 @@ 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 + // TODO: In Akka 2.1.x, ActorSystem.awaitTermination hangs when you have remote actors! + // This is unfortunate, but for now we just comment it out. workerActorSystems.foreach(_.shutdown()) - workerActorSystems.foreach(_.awaitTermination()) + //workerActorSystems.foreach(_.awaitTermination()) masterActorSystems.foreach(_.shutdown()) - masterActorSystems.foreach(_.awaitTermination()) + //masterActorSystems.foreach(_.awaitTermination()) + masterActorSystems.clear() + workerActorSystems.clear() } } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1500aee672..71b3008c8d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -96,12 +96,14 @@ object SparkBuild extends Build { */ libraryDependencies ++= Seq( - "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", - "org.easymock" % "easymock" % "3.1" % "test" - ), + "io.netty" % "netty" % "3.6.6.Final", + "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", + "org.easymock" % "easymock" % "3.1" % "test" + ), + parallelExecution := false, /* Workaround for issue #206 (fixed after SBT 0.11.0) */ watchTransitiveSources <<= Defaults.inDependencies[Task[Seq[File]]](watchSources.task, @@ -118,6 +120,9 @@ object SparkBuild extends Build { val slf4jVersion = "1.6.1" + val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson") + val excludeNetty = ExclusionRule(organization = "org.jboss.netty") + def coreSettings = sharedSettings ++ Seq( name := "spark-core", resolvers ++= Seq( @@ -134,24 +139,27 @@ object SparkBuild extends Build { "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, "com.ning" % "compress-lzf" % "0.8.4", - "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION, + "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeNetty, excludeJackson), "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.2", - "com.typesafe.akka" %% "akka-slf4j" % "2.1.2", + "com.typesafe.akka" %% "akka-remote" % "2.1.4" excludeAll(excludeNetty), + "com.typesafe.akka" %% "akka-slf4j" % "2.1.4" excludeAll(excludeNetty), "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", + "io.spray" % "spray-can" % "1.1-M7" excludeAll(excludeNetty), + "io.spray" % "spray-io" % "1.1-M7" excludeAll(excludeNetty), + "io.spray" % "spray-routing" % "1.1-M7" excludeAll(excludeNetty), + "io.spray" %% "spray-json" % "1.2.3" excludeAll(excludeNetty), "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.9.0-incubating", "org.scala-lang" % "scala-actors" % "2.10.1", "org.scala-lang" % "jline" % "2.10.1", "org.scala-lang" % "scala-reflect" % "2.10.1" ) ++ (if (HADOOP_MAJOR_VERSION == "2") - Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, + Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeNetty, excludeJackson)) + else + None + ).toSeq, unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") } ) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings @@ -175,10 +183,10 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", - "com.github.sgroschupf" % "zkclient" % "0.1", - "org.twitter4j" % "twitter4j-stream" % "3.0.3", - "com.typesafe.akka" %% "akka-zeromq" % "2.1.2" + "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty), + "com.github.sgroschupf" % "zkclient" % "0.1", + "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), + "com.typesafe.akka" %% "akka-zeromq" % "2.1.4" excludeAll(excludeNetty) ) ) ++ assemblySettings ++ extraAssemblySettings diff --git a/sbt/sbt b/sbt/sbt index 850c58e1e9..523fbb346b 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -5,4 +5,4 @@ if [ "$MESOS_HOME" != "" ]; then fi export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) export SPARK_TESTING=1 # To put test classes on classpath -java -Xmx1200m -XX:MaxPermSize=250m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" +java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" -- cgit v1.2.3 From a1e56a43b3bc7ed66cbf5d5cee9d5bc12b029f0d Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 15 Jul 2013 11:28:18 +0530 Subject: 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(+) (limited to 'core/src') diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 9817272b0e..e40c5c5c8d 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 -- cgit v1.2.3 From b59152a7c3593d31f5ecef971046b8bcfa1f2b5d Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 15 Jul 2013 12:09:17 +0530 Subject: 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(-) (limited to 'core/src') diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index 60bb0240a5..fc0c160720 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") - - } } -- cgit v1.2.3 From 63addd93a8337f5181c0e7d64c86393cb519c661 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 14:09:52 +0530 Subject: Fixed warning ClassManifest -> ClassTag --- core/src/main/scala/spark/PairRDDFunctions.scala | 2 +- core/src/main/scala/spark/RDD.scala | 6 +++--- .../main/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 +++++---- .../main/scala/spark/streaming/dstream/KafkaInputDStream.scala | 2 +- 7 files changed, 17 insertions(+), 13 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 0095b868a8..5cf162f23e 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 3a454df10d..2ebfaadc46 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 6044043add..4562884eb3 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 e4dd3a7fa7..3b9899238c 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 a50f407737..f570dd6d8b 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 b234428ab2..d3c2370885 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 e0f6351ef7..ba1312cbe8 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) { -- cgit v1.2.3 From ff14f38f3dce314eb1afb876e79cbab6b1c252b8 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 14:34:56 +0530 Subject: Fixed warning Throwables --- core/src/main/scala/spark/api/python/PythonWorkerFactory.scala | 4 ++-- .../main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 +- .../src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index 85d1dfeac8..7163f01b24 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 327d6797ae..56374be35a 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 ba1312cbe8..8b23642e4a 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 } } } -- cgit v1.2.3 From 55da6e9504c533b6eaced215c9d4a6199062f799 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 14:36:36 +0530 Subject: Fixed warning erasure -> runtimeClass --- core/src/main/scala/spark/PairRDDFunctions.scala | 2 +- .../src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 5cf162f23e..ee09c2085b 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 8b23642e4a..47274f41a5 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 -- cgit v1.2.3 From 50f3cd8890514f878a10b6b18cd4e38448d5d475 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 14:39:46 +0530 Subject: Fixed warning enumerations --- .../main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 6 +++--- core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 56374be35a..9d618cf531 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 6e0c6793e0..1c1f17ebf2 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 -- cgit v1.2.3 From f89cc7ae3c475b3ffdadbe1daccc26265f0bca2a Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 14:59:24 +0530 Subject: Fixed warning for type erasure --- core/src/main/scala/spark/api/python/PythonRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 3d1e45cb2c..d15c75289d 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) -- cgit v1.2.3 From 039087e1e3916799799b2e544288857545dddaec Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 17 Jul 2013 11:46:00 +0530 Subject: Fixed formatting As per review comments on #709 --- core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index 1c1f17ebf2..3ff3cd5fe4 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 -- cgit v1.2.3 From 276c37a51c9a6188dbbe02754935540ace338dd1 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sun, 22 Sep 2013 08:20:12 +0530 Subject: Akka 2.2 migration --- .../src/main/scala/org/apache/spark/SparkEnv.scala | 2 +- .../org/apache/spark/deploy/client/Client.scala | 10 +++---- .../org/apache/spark/deploy/master/Master.scala | 31 ++++++++++++++++------ .../org/apache/spark/deploy/worker/Worker.scala | 12 ++++----- .../scala/org/apache/spark/executor/Executor.scala | 2 +- .../spark/executor/StandaloneExecutorBackend.scala | 30 ++++++++++++++------- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../cluster/StandaloneSchedulerBackend.scala | 12 ++++----- .../mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../scala/org/apache/spark/util/AkkaUtils.scala | 23 ++++++++-------- .../scala/org/apache/spark/DistributedSuite.scala | 2 +- .../test/scala/org/apache/spark/DriverSuite.scala | 2 +- .../org/apache/spark/MapOutputTrackerSuite.scala | 2 +- .../spark/streaming/examples/ActorWordCount.scala | 2 +- .../spark/streaming/examples/ZeroMQWordCount.scala | 8 +++--- project/SparkBuild.scala | 18 ++++++------- .../apache/spark/streaming/StreamingContext.scala | 5 ++-- .../streaming/api/java/JavaStreamingContext.scala | 7 ++--- .../streaming/dstream/NetworkInputDStream.scala | 2 +- .../spark/streaming/receivers/ActorReceiver.scala | 4 +-- .../spark/streaming/receivers/ZeroMQReceiver.scala | 7 ++--- .../org/apache/spark/streaming/JavaAPISuite.java | 2 +- .../spark/deploy/yarn/YarnAllocationHandler.scala | 2 +- 23 files changed, 109 insertions(+), 80 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 1e63b54b7a..a267407c67 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -169,7 +169,7 @@ object SparkEnv extends Logging { val driverHost: String = System.getProperty("spark.driver.host", "localhost") val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt Utils.checkHost(driverHost, "Expected hostname") - val url = "akka://spark@%s:%s/user/%s".format(driverHost, driverPort, name) + val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name) logInfo("Connecting to " + name + ": " + url) actorSystem.actorFor(url) } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 14a90934f6..164386782c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -26,9 +26,7 @@ import akka.actor._ import akka.actor.Terminated import akka.pattern.AskTimeoutException import akka.pattern.ask -import akka.remote.RemoteClientDisconnected -import akka.remote.RemoteClientLifeCycleEvent -import akka.remote.RemoteClientShutdown +import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent, AssociationErrorEvent} import org.apache.spark.Logging import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} @@ -61,7 +59,7 @@ private[spark] class Client( master = context.actorFor(Master.toAkkaUrl(masterUrl)) masterAddress = master.path.address master ! RegisterApplication(appDescription) - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing } catch { case e: Exception => @@ -99,12 +97,12 @@ private[spark] class Client( markDisconnected() context.stop(self) - case RemoteClientDisconnected(transport, address) if address == masterAddress => + case DisassociatedEvent(_, address, _) if address == masterAddress => logError("Connection to master failed; stopping client") markDisconnected() context.stop(self) - case RemoteClientShutdown(transport, address) if address == masterAddress => + case AssociationErrorEvent(_, _, address, _) if address == masterAddress => logError("Connection to master failed; stopping client") markDisconnected() context.stop(self) 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 2efd16bca0..cb0fe6a850 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 @@ -25,9 +25,8 @@ import scala.concurrent.Await import scala.concurrent.duration._ import akka.actor._ -import akka.actor.Terminated import akka.pattern.ask -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} +import akka.remote._ import org.apache.spark.{Logging, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} @@ -36,6 +35,22 @@ 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 org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed +import org.apache.spark.deploy.DeployMessages.KillExecutor +import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import scala.Some +import org.apache.spark.deploy.DeployMessages.WebUIPortResponse +import org.apache.spark.deploy.DeployMessages.LaunchExecutor +import org.apache.spark.deploy.DeployMessages.RegisteredApplication +import org.apache.spark.deploy.DeployMessages.RegisterWorker +import org.apache.spark.deploy.DeployMessages.ExecutorUpdated +import org.apache.spark.deploy.DeployMessages.MasterStateResponse +import org.apache.spark.deploy.DeployMessages.ExecutorAdded +import org.apache.spark.deploy.DeployMessages.RegisterApplication +import org.apache.spark.deploy.DeployMessages.ApplicationRemoved +import org.apache.spark.deploy.DeployMessages.Heartbeat +import org.apache.spark.deploy.DeployMessages.RegisteredWorker +import akka.actor.Terminated private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { @@ -81,7 +96,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act override def preStart() { 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]) + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) webUi.start() import context.dispatcher context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut) @@ -165,13 +180,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act actorToApp.get(actor).foreach(finishApplication) } - case RemoteClientDisconnected(transport, address) => { + case DisassociatedEvent(_, address, _) => { // The disconnected client could've been either a worker or an app; remove whichever it was addressToWorker.get(address).foreach(removeWorker) addressToApp.get(address).foreach(finishApplication) } - case RemoteClientShutdown(transport, address) => { + case AssociationErrorEvent(_, _, address, _) => { // The disconnected client could've been either a worker or an app; remove whichever it was addressToWorker.get(address).foreach(removeWorker) addressToApp.get(address).foreach(finishApplication) @@ -376,11 +391,11 @@ private[spark] object Master { actorSystem.awaitTermination() } - /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ + /** Returns an `akka.tcp://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ def toAkkaUrl(sparkUrl: String): String = { sparkUrl match { case sparkUrlRegex(host, port) => - "akka://%s@%s:%s/user/%s".format(systemName, host, port, actorName) + "akka.tcp://%s@%s:%s/user/%s".format(systemName, host, port, actorName) case _ => throw new SparkException("Invalid master URL: " + sparkUrl) } @@ -388,7 +403,7 @@ 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) + val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), name = actorName) val timeoutDuration = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") implicit val timeout = Timeout(timeoutDuration) 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 a0a9d1040a..1f04c1eea5 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 @@ -25,9 +25,9 @@ import scala.collection.mutable.HashMap import scala.concurrent.duration._ import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} +import akka.remote.{RemotingLifecycleEvent, AssociationErrorEvent, DisassociatedEvent} -import org.apache.spark.{Logging} +import org.apache.spark.Logging import org.apache.spark.deploy.ExecutorState import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master @@ -113,7 +113,7 @@ private[spark] class Worker( logInfo("Connecting to master " + masterUrl) master = context.actorFor(Master.toAkkaUrl(masterUrl)) master ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get, publicAddress) - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing } @@ -165,7 +165,7 @@ private[spark] class Worker( logInfo("Asked to kill unknown executor " + fullId) } - case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => + case _: Terminated | DisassociatedEvent | AssociationErrorEvent => masterDisconnected() case RequestWorkerState => { @@ -207,8 +207,8 @@ private[spark] object Worker { // The LocalSparkCluster runs multiple local sparkWorkerX actor systems val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("") val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) - val actor = actorSystem.actorOf(Props(new Worker(host, boundPort, webUiPort, cores, memory, - masterUrl, workDir)), name = "Worker") + actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory, + masterUrl, workDir), name = "Worker") (actorSystem, boundPort) } 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 ceae3b8289..99a4a95e82 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -105,7 +105,7 @@ private[spark] class Executor( SparkEnv.set(env) env.metricsSystem.registerSource(executorSource) - private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size") // Start worker thread pool val threadPool = new ThreadPoolExecutor( 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 7839023868..46f0ef2cc6 100644 --- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala @@ -19,13 +19,25 @@ package org.apache.spark.executor import java.nio.ByteBuffer -import akka.actor.{ActorRef, Actor, Props, Terminated} -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} +import akka.actor._ +import akka.remote._ import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._ import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisteredExecutor +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.LaunchTask +import akka.remote.DisassociatedEvent +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutor +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutorFailed +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisteredExecutor +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.LaunchTask +import akka.remote.AssociationErrorEvent +import akka.remote.DisassociatedEvent +import akka.actor.Terminated +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutor +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutorFailed private[spark] class StandaloneExecutorBackend( @@ -40,14 +52,14 @@ private[spark] class StandaloneExecutorBackend( Utils.checkHostPort(hostPort, "Expected hostport") var executor: Executor = null - var driver: ActorRef = null + var driver: ActorSelection = null override def preStart() { logInfo("Connecting to driver: " + driverUrl) - driver = context.actorFor(driverUrl) + driver = context.actorSelection(driverUrl) 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 + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + // context.watch(driver) // Doesn't work with remote actors, but useful for testing } override def receive = { @@ -69,7 +81,7 @@ private[spark] class StandaloneExecutorBackend( executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask) } - case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => + case _: Terminated | DisassociatedEvent | AssociationErrorEvent => logError("Driver terminated or disconnected! Shutting down.") System.exit(1) } @@ -90,8 +102,8 @@ private[spark] object StandaloneExecutorBackend { // set it val sparkHostPort = hostname + ":" + boundPort System.setProperty("spark.hostPort", sparkHostPort) - val actor = actorSystem.actorOf( - Props(new StandaloneExecutorBackend(driverUrl, executorId, sparkHostPort, cores)), + actorSystem.actorOf( + Props(classOf[StandaloneExecutorBackend], driverUrl, executorId, sparkHostPort, cores), name = "Executor") actorSystem.awaitTermination() } 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 9c49768c0c..fa83ae19d6 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 @@ -42,7 +42,7 @@ private[spark] class SparkDeploySchedulerBackend( super.start() // The endpoint for executors to talk to us - val driverUrl = "akka://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), StandaloneSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") 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 addfa077c1..49f668eb32 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 @@ -25,7 +25,7 @@ import scala.concurrent.duration._ import akka.actor._ import akka.pattern.ask -import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} +import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent} import org.apache.spark.{SparkException, Logging, TaskState} import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._ @@ -53,7 +53,7 @@ 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]) + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) // Periodically revive offers to allow delay scheduling to work val reviveInterval = System.getProperty("spark.scheduler.revive.interval", "1000").toLong @@ -101,11 +101,11 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor case Terminated(actor) => actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated")) - case RemoteClientDisconnected(transport, address) => - addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disconnected")) + case DisassociatedEvent(_, remoteAddress, _) => + addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client disconnected")) - case RemoteClientShutdown(transport, address) => - addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client shutdown")) + case AssociationErrorEvent(_, _, remoteAddress, _) => + addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client shutdown")) } // Make fake resource offers on all executors 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 3dbe61d706..babe875fa1 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 @@ -119,7 +119,7 @@ private[spark] class CoarseMesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val driverUrl = "akka://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), StandaloneSchedulerBackend.ACTOR_NAME) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index e674d120ea..af1c36b34d 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -48,28 +48,27 @@ private[spark] object AkkaUtils { val akkaConf = ConfigFactory.parseString(""" akka.daemonic = on - akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] + akka.loggers = [""akka.event.slf4j.Slf4jLogger""] akka.stdout-loglevel = "ERROR" akka.actor.provider = "akka.remote.RemoteActorRefProvider" - akka.remote.transport = "akka.remote.netty.NettyRemoteTransport" - akka.remote.netty.hostname = "%s" - akka.remote.netty.port = %d - akka.remote.netty.connection-timeout = %ds - akka.remote.netty.message-frame-size = %d MiB - akka.remote.netty.execution-pool-size = %d + akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" + akka.remote.netty.tcp.hostname = "%s" + akka.remote.netty.tcp.port = %d + akka.remote.netty.tcp.connection-timeout = %ds + akka.remote.netty.tcp.message-frame-size = %d MiB + akka.remote.netty.tcp.execution-pool-size = %d akka.actor.default-dispatcher.throughput = %d akka.remote.log-remote-lifecycle-events = %s - akka.remote.netty.write-timeout = %ds - """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize, - lifecycleEvents, akkaWriteTimeout)) + """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize, + lifecycleEvents)) val actorSystem = ActorSystem(name, akkaConf) // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a // hack because Akka doesn't let you figure out the port through the public API yet. val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider - val boundPort = provider.asInstanceOf[RemoteActorRefProvider].transport.address.port.get - return (actorSystem, boundPort) + val boundPort = provider.getDefaultAddress.port.get + (actorSystem, boundPort) } } diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 7a856d4081..c719a54a61 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -325,7 +325,7 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter // when running under LocalScheduler: sc = new SparkContext("local-cluster[1,1,512]", "test") val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt + sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size").toInt val rdd = sc.parallelize(Seq(1)).map{x => new Array[Byte](akkaFrameSize)} val exception = intercept[SparkException] { rdd.reduce((x, y) => x) diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 01a72d8401..6d1695eae7 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -34,7 +34,7 @@ class DriverSuite extends FunSuite with Timeouts { // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing" val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => - failAfter(30 seconds) { + failAfter(60 seconds) { Utils.execute(Seq("./spark-class", "org.apache.spark.DriverWithoutCleanup", master), new File(System.getenv("SPARK_HOME"))) } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 6013320eaa..18fb1bf590 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -109,7 +109,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0) val slaveTracker = new MapOutputTracker() slaveTracker.trackerActor = slaveSystem.actorFor( - "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker") + "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker") masterTracker.registerShuffle(10, 1) masterTracker.incrementEpoch() diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 13aa24fa1a..08e399f9ee 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -165,7 +165,7 @@ object ActorWordCount { */ val lines = ssc.actorStream[String]( - Props(new SampleActorReceiver[String]("akka://test@%s:%s/user/FeederActor".format( + Props(new SampleActorReceiver[String]("akka.tcp://test@%s:%s/user/FeederActor".format( host, port.toInt))), "SampleReceiver") //compute wordcount diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index c8743b9e25..e83ce78aa5 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -23,6 +23,7 @@ import akka.zeromq._ import org.apache.spark.streaming.{ Seconds, StreamingContext } import org.apache.spark.streaming.StreamingContext._ import akka.zeromq.Subscribe +import akka.util.ByteString /** * A simple publisher for demonstration purposes, repeatedly publishes random Messages @@ -40,10 +41,11 @@ object SimpleZeroMQPublisher { val acs: ActorSystem = ActorSystem() val pubSocket = ZeroMQExtension(acs).newSocket(SocketType.Pub, Bind(url)) - val messages: Array[String] = Array("words ", "may ", "count ") + implicit def stringToByteString(x: String) = ByteString(x) + val messages: List[ByteString] = List("words ", "may ", "count ") while (true) { Thread.sleep(1000) - pubSocket ! ZMQMessage(Frame(topic) :: messages.map(x => Frame(x.getBytes)).toList) + pubSocket ! ZMQMessage(ByteString(topic) :: messages) } acs.awaitTermination() } @@ -78,7 +80,7 @@ object ZeroMQWordCount { val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - def bytesToStringIterator(x: Seq[Seq[Byte]]) = (x.map(x => new String(x.toArray))).iterator + def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator //For this stream, a zeroMQ publisher should be running. val lines = ssc.zeroMQStream(url, Subscribe(topic), bytesToStringIterator) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5e7ed81c1e..f18ebf1400 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -81,7 +81,7 @@ object SparkBuild extends Build { organization := "org.apache.spark", version := "0.8.0-SNAPSHOT", scalaVersion := "2.10.2", - scalacOptions := Seq("-unchecked", "-optimize", "-deprecation", "-target:" + SCALAC_JVM_VERSION), +// scalacOptions := Seq("-unchecked", "-optimize", "-deprecation", "-target:" + SCALAC_JVM_VERSION), javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, @@ -150,7 +150,7 @@ object SparkBuild extends Build { */ libraryDependencies ++= Seq( - "io.netty" % "netty-all" % "4.0.0.Beta2", + "io.netty" % "netty-all" % "4.0.0.CR1", "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106", "org.scalatest" %% "scalatest" % "1.9.1" % "test", "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", @@ -183,9 +183,9 @@ object SparkBuild extends Build { def coreSettings = sharedSettings ++ Seq( 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/" + // "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( @@ -200,9 +200,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-remote" % "2.1.4" excludeAll(excludeNetty), - "com.typesafe.akka" %% "akka-slf4j" % "2.1.4" excludeAll(excludeNetty), - "net.liftweb" %% "lift-json" % "2.5.1", + "com.typesafe.akka" %% "akka-remote" % "2.2.1" excludeAll(excludeNetty), + "com.typesafe.akka" %% "akka-slf4j" % "2.2.1" excludeAll(excludeNetty), + "net.liftweb" %% "lift-json" % "2.5.1" excludeAll(excludeNetty), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.12.1", @@ -271,7 +271,7 @@ object SparkBuild extends Build { "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy), "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), - "com.typesafe.akka" %% "akka-zeromq" % "2.1.4" excludeAll(excludeNetty) + "com.typesafe.akka" %% "akka-zeromq" % "2.2.1" excludeAll(excludeNetty) ) ) 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 9e14c8ace7..c722aa15ab 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -47,6 +47,7 @@ import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path import twitter4j.Status import twitter4j.auth.Authorization +import akka.util.ByteString /** @@ -231,11 +232,11 @@ class StreamingContext private ( def zeroMQStream[T: ClassTag]( publisherUrl:String, subscribe: Subscribe, - bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T], + bytesToObjects: Seq[ByteString] ⇒ Iterator[T], storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2, supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy ): DStream[T] = { - actorStream(Props(new ZeroMQReceiver(publisherUrl,subscribe,bytesToObjects)), + actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)), "ZeroMQReceiver", storageLevel, supervisorStrategy) } 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 8135d2499e..8242af6d5f 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 @@ -29,6 +29,7 @@ import twitter4j.Status import akka.actor.Props import akka.actor.SupervisorStrategy import akka.zeromq.Subscribe +import akka.util.ByteString import twitter4j.auth.Authorization @@ -475,7 +476,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { def zeroMQStream[T]( publisherUrl:String, subscribe: Subscribe, - bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T], + bytesToObjects: Seq[ByteString] ⇒ Iterator[T], storageLevel: StorageLevel, supervisorStrategy: SupervisorStrategy ): JavaDStream[T] = { @@ -502,7 +503,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { ): JavaDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel) } @@ -522,7 +523,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { ): JavaDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator ssc.zeroMQStream[T](publisherUrl, subscribe, fn) } 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 a61a1780f1..394a39fbb0 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 @@ -177,7 +177,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging logInfo("Attempting to register with tracker") val ip = System.getProperty("spark.driver.host", "localhost") val port = System.getProperty("spark.driver.port", "7077").toInt - val url = "akka://spark@%s:%s/user/NetworkInputTracker".format(ip, port) + val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) val tracker = env.actorSystem.actorFor(url) val timeout = 5.seconds diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala index c220127c00..ee087a1cf0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala @@ -51,7 +51,7 @@ object ReceiverSupervisorStrategy { * @example {{{ * class MyActor extends Actor with Receiver{ * def receive { - * case anything :String ⇒ pushBlock(anything) + * case anything :String => pushBlock(anything) * } * } * //Can be plugged in actorStream as follows @@ -121,7 +121,7 @@ private[streaming] class ActorReceiver[T: ClassTag]( protected lazy val supervisor = env.actorSystem.actorOf(Props(new Supervisor), "Supervisor" + streamId) - private class Supervisor extends Actor { + class Supervisor extends Actor { override val supervisorStrategy = receiverSupervisorStrategy val worker = context.actorOf(props, name) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala index e009325b67..ce8c56fa8a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.receivers import akka.actor.Actor +import akka.util.ByteString import akka.zeromq._ import org.apache.spark.Logging @@ -29,7 +30,7 @@ import scala.reflect.ClassTag */ private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, subscribe: Subscribe, - bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T]) + bytesToObjects: Seq[ByteString] ⇒ Iterator[T]) extends Actor with Receiver with Logging { override def preStart() = ZeroMQExtension(context.system).newSocket(SocketType.Sub, Listener(self), @@ -40,10 +41,10 @@ private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, case Connecting ⇒ logInfo("connecting ...") case m: ZMQMessage ⇒ - logDebug("Received message for:" + m.firstFrameAsString) + logDebug("Received message for:" + m.frame(0)) //We ignore first frame for processing as it is the topic - val bytes = m.frames.tail.map(_.payload) + val bytes = m.frames.tail pushBlock(bytesToObjects(bytes)) case Closed ⇒ logInfo("received closed ") diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index c0d729ff87..783b8dea31 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -48,7 +48,7 @@ import java.util.*; import akka.actor.Props; import akka.zeromq.Subscribe; - +import akka.util.ByteString; // The test suite itself is Serializable so that anonymous Function implementations can be diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 6d6ef149cc..d222f412a0 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -209,7 +209,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM else { // deallocate + allocate can result in reusing id's wrongly - so use a different counter (workerIdCounter) val workerId = workerIdCounter.incrementAndGet().toString - val driverUrl = "akka://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), StandaloneSchedulerBackend.ACTOR_NAME) -- cgit v1.2.3 From dfbdc9ddb773e2b1149e6a6c661f14b631b692d0 Mon Sep 17 00:00:00 2001 From: shane-huang Date: Mon, 23 Sep 2013 11:28:58 +0800 Subject: added spark-class and spark-executor to sbin Signed-off-by: shane-huang --- assembly/src/main/assembly/assembly.xml | 11 +- bin/spark-daemon.sh | 2 +- .../mesos/CoarseMesosSchedulerBackend.scala | 4 +- .../scheduler/mesos/MesosSchedulerBackend.scala | 4 +- .../org/apache/spark/ui/UIWorkloadGenerator.scala | 2 +- .../test/scala/org/apache/spark/DriverSuite.scala | 2 +- docs/running-on-yarn.md | 4 +- docs/spark-standalone.md | 2 +- make-distribution.sh | 5 +- python/pyspark/java_gateway.py | 2 +- sbin/spark-class | 117 +++++++++++++++++++++ sbin/spark-class.cmd | 23 ++++ sbin/spark-class2.cmd | 78 ++++++++++++++ sbin/spark-executor | 22 ++++ spark-class | 117 --------------------- spark-class.cmd | 23 ---- spark-class2.cmd | 78 -------------- spark-executor | 22 ---- 18 files changed, 256 insertions(+), 262 deletions(-) create mode 100755 sbin/spark-class create mode 100644 sbin/spark-class.cmd create mode 100644 sbin/spark-class2.cmd create mode 100755 sbin/spark-executor delete mode 100755 spark-class delete mode 100644 spark-class.cmd delete mode 100644 spark-class2.cmd delete mode 100755 spark-executor (limited to 'core/src') diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml index 47d3fa93d0..6af383db65 100644 --- a/assembly/src/main/assembly/assembly.xml +++ b/assembly/src/main/assembly/assembly.xml @@ -39,23 +39,20 @@ - ${project.parent.basedir}/bin/ + ${project.parent.basedir}/sbin/ - /bin + /sbin **/* - ${project.parent.basedir} + ${project.parent.basedir}/bin/ /bin - run-example* - spark-class* - spark-shell* - spark-executor* + **/* diff --git a/bin/spark-daemon.sh b/bin/spark-daemon.sh index 5bfe967fbf..6baee0c33b 100755 --- a/bin/spark-daemon.sh +++ b/bin/spark-daemon.sh @@ -128,7 +128,7 @@ case $startStop in 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 & + nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/sbin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & newpid=$! echo $newpid > $pid sleep 2 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 3dbe61d706..9f93491e5a 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 @@ -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, "spark-class").getCanonicalPath + val runScript = new File(sparkHome, "/sbin/spark-class").getCanonicalPath command.setValue( "\"%s\" org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format( runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) @@ -134,7 +134,7 @@ private[spark] class CoarseMesosSchedulerBackend( // glob the directory "correctly". val basename = uri.split('/').last.split('.').head command.setValue( - "cd %s*; ./spark-class org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format( + "cd %s*; ./sbin/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)) } 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 541f86e338..c7804a3f94 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 @@ -101,12 +101,12 @@ private[spark] class MesosSchedulerBackend( .setEnvironment(environment) val uri = System.getProperty("spark.executor.uri") if (uri == null) { - command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath) + command.setValue(new File(sparkHome, "/sbin/spark-executor").getCanonicalPath) } 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-executor".format(basename)) + command.setValue("cd %s*; ./sbin/spark-executor".format(basename)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } val memory = Resource.newBuilder() diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index 3ec9760ed0..b76ad673a5 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -35,7 +35,7 @@ private[spark] object UIWorkloadGenerator { def main(args: Array[String]) { if (args.length < 2) { - println("usage: ./spark-class spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") + println("usage: ./sbin/spark-class spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") System.exit(1) } val master = args(0) diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 01a72d8401..5e5ece0033 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -35,7 +35,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", "org.apache.spark.DriverWithoutCleanup", master), + Utils.execute(Seq("./sbin/spark-class", "org.apache.spark.DriverWithoutCleanup", master), new File(System.getenv("SPARK_HOME"))) } } diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index c611db0af4..767eb5cdac 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -42,7 +42,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= ./spark-class org.apache.spark.deploy.yarn.Client \ + SPARK_JAR= ./sbin/spark-class org.apache.spark.deploy.yarn.Client \ --jar \ --class \ --args \ @@ -62,7 +62,7 @@ For example: # Submit Spark's ApplicationMaster to YARN's ResourceManager, and instruct Spark to run the SparkPi example $ SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - ./spark-class org.apache.spark.deploy.yarn.Client \ + ./sbin/spark-class org.apache.spark.deploy.yarn.Client \ --jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ --class org.apache.spark.examples.SparkPi \ --args yarn-standalone \ diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 81cdbefd0c..9d4ad1ec8d 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -25,7 +25,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 org.apache.spark.deploy.worker.Worker spark://IP:PORT + ./sbin/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/make-distribution.sh b/make-distribution.sh index bffb19843c..13843bb6c2 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -98,10 +98,7 @@ mkdir "$DISTDIR"/conf cp "$FWDIR/conf/*.template" "$DISTDIR"/conf cp -r "$FWDIR/bin" "$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" +cp -r "$FWDIR/sbin" "$DISTDIR" if [ "$MAKE_TGZ" == "true" ]; then diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index e615c1e9b6..f7834ef803 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -31,7 +31,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 on_windows = platform.system() == "Windows" - script = "spark-class.cmd" if on_windows else "spark-class" + script = "/sbin/spark-class.cmd" if on_windows else "/sbin/spark-class" command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer", "--die-on-broken-pipe", "0"] if not on_windows: diff --git a/sbin/spark-class b/sbin/spark-class new file mode 100755 index 0000000000..e111ef6da7 --- /dev/null +++ b/sbin/spark-class @@ -0,0 +1,117 @@ +#!/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: spark-class []" >&2 + exit 1 +fi + +# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable +# values for that; it doesn't need a lot +if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then + SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} + SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" + # 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 + + +# Add java opts for master, worker, executor. The opts maybe null +case "$1" in + 'org.apache.spark.deploy.master.Master') + OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS" + ;; + 'org.apache.spark.deploy.worker.Worker') + OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS" + ;; + 'org.apache.spark.executor.StandaloneExecutorBackend') + OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS" + ;; + 'org.apache.spark.executor.MesosExecutorBackend') + OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS" + ;; + 'org.apache.spark.repl.Main') + OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS" + ;; +esac + +# Find the 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 + +# 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 +JAVA_OPTS="$OUR_JAVA_OPTS" +JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" +JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM" +# Load extra JAVA_OPTS from conf/java-opts, if it exists +if [ -e $FWDIR/conf/java-opts ] ; then + JAVA_OPTS="$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! + +if [ ! -f "$FWDIR/RELEASE" ]; then + # Exit if the user hasn't compiled Spark + 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 build Spark with sbt/sbt assembly before running this program" >&2 + exit 1 + fi +fi + +# Compute classpath using external script +CLASSPATH=`$FWDIR/bin/compute-classpath.sh` +export CLASSPATH + +if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then + echo -n "Spark Command: " + echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" + echo "========================================" + echo +fi + +exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" diff --git a/sbin/spark-class.cmd b/sbin/spark-class.cmd new file mode 100644 index 0000000000..19850db9e1 --- /dev/null +++ b/sbin/spark-class.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 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/sbin/spark-class2.cmd b/sbin/spark-class2.cmd new file mode 100644 index 0000000000..d4d853e8ad --- /dev/null +++ b/sbin/spark-class2.cmd @@ -0,0 +1,78 @@ +@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: spark-class ^ [^] + goto exit +:arg_given + +set RUNNING_DAEMON=0 +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 +set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true +if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY% +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 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 Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! + +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 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/sbin/spark-executor b/sbin/spark-executor new file mode 100755 index 0000000000..2c07c54843 --- /dev/null +++ b/sbin/spark-executor @@ -0,0 +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/spark-class org.apache.spark.executor.MesosExecutorBackend diff --git a/spark-class b/spark-class deleted file mode 100755 index e111ef6da7..0000000000 --- a/spark-class +++ /dev/null @@ -1,117 +0,0 @@ -#!/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: spark-class []" >&2 - exit 1 -fi - -# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable -# values for that; it doesn't need a lot -if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then - SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} - SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" - # 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 - - -# Add java opts for master, worker, executor. The opts maybe null -case "$1" in - 'org.apache.spark.deploy.master.Master') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS" - ;; - 'org.apache.spark.deploy.worker.Worker') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS" - ;; - 'org.apache.spark.executor.StandaloneExecutorBackend') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS" - ;; - 'org.apache.spark.executor.MesosExecutorBackend') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS" - ;; - 'org.apache.spark.repl.Main') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS" - ;; -esac - -# Find the 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 - -# 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 -JAVA_OPTS="$OUR_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" -JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM" -# Load extra JAVA_OPTS from conf/java-opts, if it exists -if [ -e $FWDIR/conf/java-opts ] ; then - JAVA_OPTS="$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! - -if [ ! -f "$FWDIR/RELEASE" ]; then - # Exit if the user hasn't compiled Spark - 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 build Spark with sbt/sbt assembly before running this program" >&2 - exit 1 - fi -fi - -# Compute classpath using external script -CLASSPATH=`$FWDIR/bin/compute-classpath.sh` -export CLASSPATH - -if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then - echo -n "Spark Command: " - echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" - echo "========================================" - echo -fi - -exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" diff --git a/spark-class.cmd b/spark-class.cmd deleted file mode 100644 index 19850db9e1..0000000000 --- a/spark-class.cmd +++ /dev/null @@ -1,23 +0,0 @@ -@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 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/spark-class2.cmd b/spark-class2.cmd deleted file mode 100644 index d4d853e8ad..0000000000 --- a/spark-class2.cmd +++ /dev/null @@ -1,78 +0,0 @@ -@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: spark-class ^ [^] - goto exit -:arg_given - -set RUNNING_DAEMON=0 -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 -set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true -if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY% -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 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 Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! - -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 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/spark-executor b/spark-executor deleted file mode 100755 index 2c07c54843..0000000000 --- a/spark-executor +++ /dev/null @@ -1,22 +0,0 @@ -#!/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/spark-class org.apache.spark.executor.MesosExecutorBackend -- cgit v1.2.3 From 14098037630196d2672431539503f27be67be480 Mon Sep 17 00:00:00 2001 From: shane-huang Date: Thu, 26 Sep 2013 12:50:10 +0800 Subject: fix path Signed-off-by: shane-huang --- core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core/src') 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 e3dc30eefc..d48545ecc1 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 @@ -120,7 +120,7 @@ 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), + Seq(sparkHome + "/sbin/compute-classpath" + ext), extraEnvironment=appDesc.command.environment) Seq("-cp", classPath) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts -- cgit v1.2.3 From 42f30b5590b229b2f672e7dbc4485fbf58bfb999 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 26 Sep 2013 14:38:42 +0530 Subject: Fixed UISuite, for case when port 4040 is already bound on machine running the test. --- core/src/test/scala/org/apache/spark/ui/UISuite.scala | 3 --- 1 file changed, 3 deletions(-) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 07c9f2382b..f4e1d4e802 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -25,11 +25,8 @@ import org.eclipse.jetty.server.Server class UISuite extends FunSuite { test("jetty port increases under contention") { val startPort = 4040 - val server = new Server(startPort) - server.start() val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("localhost", startPort, Seq()) val (jettyServer2, boundPort2) = JettyUtils.startJettyServer("localhost", startPort, Seq()) - // 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) -- cgit v1.2.3 From e8b1ee04fcb4cd7b666d3148d6d5ff148551ce72 Mon Sep 17 00:00:00 2001 From: shane-huang Date: Thu, 26 Sep 2013 17:08:47 +0800 Subject: fix paths and change spark to use APP_MEM as application driver memory instead of SPARK_MEM, user should add application jars to SPARK_CLASSPATH Signed-off-by: shane-huang --- bin/spark | 41 +++++----------------- .../mesos/CoarseMesosSchedulerBackend.scala | 2 +- python/pyspark/java_gateway.py | 2 +- 3 files changed, 10 insertions(+), 35 deletions(-) (limited to 'core/src') diff --git a/bin/spark b/bin/spark index 7f25fe1050..aa005a51f5 100755 --- a/bin/spark +++ b/bin/spark @@ -31,40 +31,11 @@ if [ -e $FWDIR/conf/spark-env.sh ] ; then fi if [ -z "$1" ]; then - echo "Usage: spark-class []" >&2 + echo "Usage: spark []" >&2 + echo "Usage: export SPARK_CLASSPATH before running the command" >&2 exit 1 fi -# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable -# values for that; it doesn't need a lot -if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then - SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} - SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" - # 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 - - -# Add java opts for master, worker, executor. The opts maybe null -case "$1" in - 'org.apache.spark.deploy.master.Master') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS" - ;; - 'org.apache.spark.deploy.worker.Worker') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS" - ;; - 'org.apache.spark.executor.StandaloneExecutorBackend') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS" - ;; - 'org.apache.spark.executor.MesosExecutorBackend') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS" - ;; - 'org.apache.spark.repl.Main') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS" - ;; -esac # Find the java binary if [ -n "${JAVA_HOME}" ]; then @@ -78,14 +49,18 @@ else fi fi -# Set SPARK_MEM if it isn't already set since we also use it for this process +# Set SPARK_MEM if it isn't already set SPARK_MEM=${SPARK_MEM:-512m} export SPARK_MEM +# Set APP_MEM if it isn't already set, we use this for this process as the app driver process may need +# as much memory as specified in SPARK_MEM +APP_MEM=${APP_MEM:-512m} + # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$OUR_JAVA_OPTS" JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" -JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM" +JAVA_OPTS="$JAVA_OPTS -Xms$APP_MEM -Xmx$APP_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e $FWDIR/conf/java-opts ] ; then JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" 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 9f93491e5a..544b20550e 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 @@ -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, "/sbin/spark-class").getCanonicalPath + val runScript = new File(sparkHome, "./sbin/spark-class").getCanonicalPath command.setValue( "\"%s\" org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format( runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index f7834ef803..b872ae61d5 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -31,7 +31,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 on_windows = platform.system() == "Windows" - script = "/sbin/spark-class.cmd" if on_windows else "/sbin/spark-class" + script = "./sbin/spark-class.cmd" if on_windows else "./sbin/spark-class" command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer", "--die-on-broken-pipe", "0"] if not on_windows: -- cgit v1.2.3 From 9865fd6aa07fd722c6782ec8001b732c39a98d45 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 30 Sep 2013 18:09:12 +0530 Subject: Fixed non termination of Executor backend, when sc.stop is not called. --- core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 2 +- .../scala/org/apache/spark/executor/StandaloneExecutorBackend.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') 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 1f04c1eea5..3904b701b2 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 @@ -165,7 +165,7 @@ private[spark] class Worker( logInfo("Asked to kill unknown executor " + fullId) } - case _: Terminated | DisassociatedEvent | AssociationErrorEvent => + case DisassociatedEvent(_, _, _) => masterDisconnected() case RequestWorkerState => { 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 46f0ef2cc6..f705a5631a 100644 --- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala @@ -81,7 +81,7 @@ private[spark] class StandaloneExecutorBackend( executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask) } - case _: Terminated | DisassociatedEvent | AssociationErrorEvent => + case DisassociatedEvent(_, _, _) => logError("Driver terminated or disconnected! Shutting down.") System.exit(1) } -- cgit v1.2.3 From 3e41495288c897ee3d3b31d5f4836b1cf6ba54d9 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sat, 5 Oct 2013 16:39:25 +0530 Subject: Fixed tests, changed property akka.remote.netty.x to akka.remote.netty.tcp.x --- .../org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala index 119ba30090..370a3eb0eb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala @@ -81,7 +81,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA test("handling results larger than Akka frame size") { val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt + sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) assert(result === 1.to(akkaFrameSize).toArray) @@ -102,7 +102,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA } scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt + sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) assert(result === 1.to(akkaFrameSize).toArray) -- cgit v1.2.3 From e09f4a9601b18921c309903737d309eab5c6d891 Mon Sep 17 00:00:00 2001 From: Martin Weindel Date: Sat, 5 Oct 2013 23:08:23 +0200 Subject: fixed some warnings --- .../org/apache/spark/network/netty/FileClient.java | 2 - .../org/apache/spark/network/netty/FileServer.java | 1 - .../java/JavaSparkContextVarargsWorkaround.java | 1 - .../spark/api/java/function/DoubleFunction.java | 2 - .../apache/spark/api/java/function/Function.java | 1 - .../apache/spark/api/java/function/Function2.java | 1 - .../api/java/function/PairFlatMapFunction.java | 1 - .../spark/api/java/function/PairFunction.java | 1 - .../spark/api/python/PythonWorkerFactory.scala | 4 +- .../org/apache/spark/deploy/ExecutorState.scala | 2 +- .../spark/deploy/master/ApplicationState.scala | 2 +- .../apache/spark/deploy/master/WorkerState.scala | 2 +- .../main/scala/org/apache/spark/rdd/BlockRDD.scala | 3 +- .../scala/org/apache/spark/rdd/CoalescedRDD.scala | 3 +- .../main/scala/org/apache/spark/rdd/EmptyRDD.scala | 3 +- .../scala/org/apache/spark/rdd/FilteredRDD.scala | 3 +- .../scala/org/apache/spark/rdd/FlatMappedRDD.scala | 3 +- .../scala/org/apache/spark/rdd/GlommedRDD.scala | 3 +- .../org/apache/spark/rdd/MapPartitionsRDD.scala | 3 +- .../spark/rdd/MapPartitionsWithIndexRDD.scala | 3 +- .../scala/org/apache/spark/rdd/MappedRDD.scala | 3 +- .../org/apache/spark/rdd/OrderedRDDFunctions.scala | 7 +- .../org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- .../scala/org/apache/spark/rdd/ShuffledRDD.scala | 3 +- .../scheduler/cluster/ClusterTaskSetManager.scala | 2 +- .../spark/scheduler/cluster/SchedulingMode.scala | 2 +- .../spark/scheduler/cluster/TaskLocality.scala | 2 +- .../scala/org/apache/spark/DistributedSuite.scala | 2 +- .../scala/org/apache/spark/UnpersistSuite.scala | 2 +- .../test/scala/org/apache/spark/rdd/RDDSuite.scala | 8 +- .../org/apache/spark/examples/JavaLogQuery.java | 2 +- .../org/apache/spark/examples/JavaPageRank.java | 3 +- .../org/apache/spark/examples/JavaWordCount.java | 2 +- .../org/apache/spark/mllib/examples/JavaALS.java | 1 - .../spark/mllib/clustering/JavaKMeansSuite.java | 4 +- .../spark/mllib/recommendation/JavaALSSuite.java | 2 - .../spark/streaming/dstream/CoGroupedDStream.scala | 3 +- .../streaming/dstream/ConstantInputDStream.scala | 3 +- .../spark/streaming/dstream/FilteredDStream.scala | 3 +- .../streaming/dstream/FlatMapValuedDStream.scala | 3 +- .../streaming/dstream/FlatMappedDStream.scala | 3 +- .../spark/streaming/dstream/ForEachDStream.scala | 3 +- .../spark/streaming/dstream/GlommedDStream.scala | 3 +- .../streaming/dstream/MapPartitionedDStream.scala | 3 +- .../spark/streaming/dstream/MapValuedDStream.scala | 3 +- .../spark/streaming/dstream/MappedDStream.scala | 3 +- .../streaming/dstream/PluggableInputDStream.scala | 3 +- .../streaming/dstream/QueueInputDStream.scala | 4 +- .../spark/streaming/dstream/ShuffledDStream.scala | 3 +- .../streaming/dstream/TransformedDStream.scala | 3 +- .../org/apache/spark/streaming/JavaAPISuite.java | 94 +++++++++++----------- 51 files changed, 120 insertions(+), 108 deletions(-) (limited to 'core/src') diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java index 20a7a3aa8c..edd0fc56f8 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClient.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClient.java @@ -19,8 +19,6 @@ package org.apache.spark.network.netty; import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelOption; import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.oio.OioSocketChannel; diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServer.java b/core/src/main/java/org/apache/spark/network/netty/FileServer.java index 666432474d..a99af348ce 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServer.java @@ -20,7 +20,6 @@ package org.apache.spark.network.netty; import java.net.InetSocketAddress; import io.netty.bootstrap.ServerBootstrap; -import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelOption; import io.netty.channel.oio.OioEventLoopGroup; diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java index c9cbce5624..2090efd3b9 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java @@ -17,7 +17,6 @@ package org.apache.spark.api.java; -import java.util.Arrays; import java.util.ArrayList; import java.util.List; diff --git a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java index db34cd190a..ed92d31af5 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java @@ -18,8 +18,6 @@ package org.apache.spark.api.java.function; -import scala.runtime.AbstractFunction1; - import java.io.Serializable; /** diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function.java b/core/src/main/scala/org/apache/spark/api/java/function/Function.java index f9dae6ed34..e97116986f 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function.java @@ -19,7 +19,6 @@ package org.apache.spark.api.java.function; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import scala.runtime.AbstractFunction1; import java.io.Serializable; diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java index 1659bfc552..cf77bb6b73 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java @@ -19,7 +19,6 @@ package org.apache.spark.api.java.function; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import scala.runtime.AbstractFunction2; import java.io.Serializable; diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java index 5a5c9b6296..fbd0cdabe0 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java @@ -20,7 +20,6 @@ package org.apache.spark.api.java.function; import scala.Tuple2; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import scala.runtime.AbstractFunction1; import java.io.Serializable; diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java index 4c39f483e5..f09559627d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java @@ -20,7 +20,6 @@ package org.apache.spark.api.java.function; import scala.Tuple2; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import scala.runtime.AbstractFunction1; import java.io.Serializable; 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 67d45723ba..f291266fcf 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 @@ -64,7 +64,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String startDaemon() new Socket(daemonHost, daemonPort) } - case e => throw e + case e: Throwable => throw e } } } @@ -198,7 +198,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/org/apache/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala index fcfea96ad6..72f84c99fc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy private[spark] object ExecutorState - extends Enumeration("LAUNCHING", "LOADING", "RUNNING", "KILLED", "FAILED", "LOST") { + extends Enumeration { val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala index 7e804223cf..39ef090ddf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.master private[spark] object ApplicationState - extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") { + extends Enumeration { type ApplicationState = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala index b5ee6dca79..fb3fe88d92 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object WorkerState extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED") { +private[spark] object WorkerState extends Enumeration { type WorkerState = Value val ALIVE, DEAD, DECOMMISSIONED = Value 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 bca6956a18..fe2946bcbe 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -19,13 +19,14 @@ package org.apache.spark.rdd import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} import org.apache.spark.storage.BlockManager +import scala.reflect.ClassTag private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition { val index = idx } private[spark] -class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String]) +class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[String]) extends RDD[T](sc, Nil) { @transient lazy val locations_ = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index c5de6362a9..98da35763b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -22,6 +22,7 @@ import java.io.{ObjectOutputStream, IOException} import scala.collection.mutable import scala.Some import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag /** * Class that captures a coalesced RDD by essentially keeping track of parent partitions @@ -68,7 +69,7 @@ case class CoalescedRDDPartition( * @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]( +class CoalescedRDD[T: ClassTag]( @transient var prev: RDD[T], maxPartitions: Int, balanceSlack: Double = 0.10) 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 c8900d1a93..99c34c6cc5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -18,12 +18,13 @@ package org.apache.spark.rdd import org.apache.spark.{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/org/apache/spark/rdd/FilteredRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala index 5312dc0b59..e74c83b90b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala @@ -18,8 +18,9 @@ package org.apache.spark.rdd import org.apache.spark.{OneToOneDependency, Partition, TaskContext} +import scala.reflect.ClassTag -private[spark] class FilteredRDD[T: ClassManifest]( +private[spark] class FilteredRDD[T: ClassTag]( prev: RDD[T], f: T => Boolean) extends RDD[T](prev) { 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 cbdf6d84c0..4d1878fc14 100644 --- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala @@ -18,10 +18,11 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} +import scala.reflect.ClassTag private[spark] -class FlatMappedRDD[U: ClassManifest, T: ClassManifest]( +class FlatMappedRDD[U: ClassTag, T: ClassTag]( prev: RDD[T], f: T => TraversableOnce[U]) extends RDD[U](prev) { 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 829545d7b0..1a694475f6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala @@ -18,8 +18,9 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} +import scala.reflect.ClassTag -private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T]) +private[spark] class GlommedRDD[T: ClassTag](prev: RDD[T]) extends RDD[Array[T]](prev) { override def getPartitions: Array[Partition] = firstParent[T].partitions 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 203179c4ea..cdb5946b49 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -18,10 +18,11 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} +import scala.reflect.ClassTag private[spark] -class MapPartitionsRDD[U: ClassManifest, T: ClassManifest]( +class MapPartitionsRDD[U: ClassTag, T: ClassTag]( prev: RDD[T], f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false) 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 3ed8339010..3cf22851dd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} +import scala.reflect.ClassTag /** @@ -26,7 +27,7 @@ import org.apache.spark.{Partition, TaskContext} * information such as the number of tuples in a partition. */ private[spark] -class MapPartitionsWithIndexRDD[U: ClassManifest, T: ClassManifest]( +class MapPartitionsWithIndexRDD[U: ClassTag, T: ClassTag]( prev: RDD[T], f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean 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 e8be1c4816..eb3b19907d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala @@ -18,9 +18,10 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} +import scala.reflect.ClassTag private[spark] -class MappedRDD[U: ClassManifest, T: ClassManifest](prev: RDD[T], f: T => U) +class MappedRDD[U: ClassTag, T: ClassTag](prev: RDD[T], f: T => U) extends RDD[U](prev) { override def getPartitions: Array[Partition] = firstParent[T].partitions 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 697be8b997..4a465840c6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -18,6 +18,7 @@ package org.apache.spark.rdd import org.apache.spark.{RangePartitioner, Logging} +import scala.reflect.ClassTag /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through @@ -25,9 +26,9 @@ import org.apache.spark.{RangePartitioner, Logging} * 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, - P <: Product2[K, V] : ClassManifest]( +class OrderedRDDFunctions[K <% Ordered[K]: ClassTag, + V: ClassTag, + P <: Product2[K, V] : ClassTag]( self: RDD[P]) extends Logging with Serializable { diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index aed585e6a1..c8e623081a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -51,7 +51,7 @@ import org.apache.spark.Partitioner.defaultPartitioner * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. * 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)]) +class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) extends Logging with SparkHadoopMapReduceUtil with Serializable { 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 9537152335..b7205865cf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.rdd import org.apache.spark.{Dependency, Partitioner, SparkEnv, ShuffleDependency, Partition, TaskContext} +import scala.reflect.ClassTag private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -32,7 +33,7 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { * @tparam K the key class. * @tparam V the value class. */ -class ShuffledRDD[K, V, P <: Product2[K, V] : ClassManifest]( +class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( @transient var prev: RDD[P], part: Partitioner) extends RDD[P](prev.context, Nil) { 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 0ac3d7bcfd..f61fde6957 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 @@ -461,7 +461,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/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala index 34811389a0..16013b3208 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala @@ -22,7 +22,7 @@ package org.apache.spark.scheduler.cluster * to order tasks amongst a Schedulable's sub-queues * "NONE" is used when the a Schedulable has no sub-queues. */ -object SchedulingMode extends Enumeration("FAIR", "FIFO", "NONE") { +object SchedulingMode extends Enumeration { type SchedulingMode = Value val FAIR,FIFO,NONE = Value diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala index 5d4130e14a..8d8d708612 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster private[spark] object TaskLocality - extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") + extends Enumeration { // process local is expected to be used ONLY within tasksetmanager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index c719a54a61..adc6ca94ff 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -313,7 +313,7 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter Thread.sleep(200) } } catch { - case _ => { Thread.sleep(10) } + case _: Throwable => { Thread.sleep(10) } // Do nothing. We might see exceptions because block manager // is racing this thread to remove entries from the driver. } diff --git a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala index 46a2da1724..768ca3850e 100644 --- a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala +++ b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala @@ -37,7 +37,7 @@ class UnpersistSuite extends FunSuite with LocalSparkContext { Thread.sleep(200) } } catch { - case _ => { Thread.sleep(10) } + case _: Throwable => { Thread.sleep(10) } // Do nothing. We might see exceptions because block manager // is racing this thread to remove entries from the driver. } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index c1df5e151e..096023f476 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -225,8 +225,8 @@ 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(1.)((perc, loc) => math.min(perc,loc)) - assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.).toInt + "%") + .foldLeft(1.0)((perc, loc) => math.min(perc,loc)) + assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.0).toInt + "%") // test that the groups are load balanced with 100 +/- 20 elements in each val maxImbalance = coalesced2.partitions @@ -238,9 +238,9 @@ class RDDSuite extends FunSuite with SharedSparkContext { val coalesced3 = data3.coalesce(numMachines*2) val minLocality2 = coalesced3.partitions .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) - .foldLeft(1.)((perc, loc) => math.min(perc,loc)) + .foldLeft(1.0)((perc, loc) => math.min(perc,loc)) assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + - (minLocality2*100.).toInt + "%") + (minLocality2*100.0).toInt + "%") } test("zipped RDDs") { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index 152f029213..407cd7ccfa 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -123,7 +123,7 @@ public class JavaLogQuery { }); List, Stats>> output = counts.collect(); - for (Tuple2 t : output) { + for (Tuple2 t : output) { System.out.println(t._1 + "\t" + t._2); } System.exit(0); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index c5603a639b..89aed8f279 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -21,7 +21,6 @@ import scala.Tuple2; 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; @@ -106,7 +105,7 @@ public class JavaPageRank { // Collects all URL ranks and dump them to console. List> output = ranks.collect(); - for (Tuple2 tuple : output) { + for (Tuple2 tuple : output) { System.out.println(tuple._1 + " has rank: " + tuple._2 + "."); } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index 07d32ad659..bd6383e13d 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -58,7 +58,7 @@ public class JavaWordCount { }); List> output = counts.collect(); - for (Tuple2 tuple : output) { + for (Tuple2 tuple : output) { System.out.println(tuple._1 + ": " + tuple._2); } System.exit(0); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java index 628cb892b6..45a0d237da 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java @@ -25,7 +25,6 @@ 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; import java.util.StringTokenizer; diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java index 32d3934ac1..33b99f4bd3 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java @@ -77,7 +77,7 @@ public class JavaKMeansSuite implements Serializable { @Test public void runKMeansUsingStaticMethods() { - List points = new ArrayList(); + List points = new ArrayList(); points.add(new double[]{1.0, 2.0, 6.0}); points.add(new double[]{1.0, 3.0, 0.0}); points.add(new double[]{1.0, 4.0, 6.0}); @@ -94,7 +94,7 @@ public class JavaKMeansSuite implements Serializable { @Test public void runKMeansUsingConstructor() { - List points = new ArrayList(); + List points = new ArrayList(); points.add(new double[]{1.0, 2.0, 6.0}); points.add(new double[]{1.0, 3.0, 0.0}); points.add(new double[]{1.0, 4.0, 6.0}); diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java index 3323f6cee2..c474e01188 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java @@ -20,8 +20,6 @@ package org.apache.spark.mllib.recommendation; import java.io.Serializable; import java.util.List; -import scala.Tuple2; - import org.junit.After; import org.junit.Assert; import org.junit.Before; 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 4eddc755b9..16c1567355 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 @@ -21,9 +21,10 @@ 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} +import scala.reflect.ClassTag private[streaming] -class CoGroupedDStream[K : ClassManifest]( +class CoGroupedDStream[K : ClassTag]( parents: Seq[DStream[(K, _)]], partitioner: Partitioner ) extends DStream[(K, Seq[Seq[_]])](parents.head.ssc) { 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 a9a05c9981..f396c34758 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 @@ -19,11 +19,12 @@ package org.apache.spark.streaming.dstream import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Time, StreamingContext} +import scala.reflect.ClassTag /** * An input stream that always returns the same RDD on each timestep. Useful for testing. */ -class ConstantInputDStream[T: ClassManifest](ssc_ : StreamingContext, rdd: RDD[T]) +class ConstantInputDStream[T: ClassTag](ssc_ : StreamingContext, rdd: RDD[T]) extends InputDStream[T](ssc_) { override def start() {} 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 91ee2c1a36..db2e0a4cee 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 @@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag private[streaming] -class FilteredDStream[T: ClassManifest]( +class FilteredDStream[T: ClassTag]( parent: DStream[T], filterFunc: T => Boolean ) extends DStream[T](parent.ssc) { 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 ca7d7ca49e..244dc3ee4f 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 @@ -20,9 +20,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ +import scala.reflect.ClassTag private[streaming] -class FlatMapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest]( +class FlatMapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag]( parent: DStream[(K, V)], flatMapValueFunc: V => TraversableOnce[U] ) extends DStream[(K, U)](parent.ssc) { 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 b37966f9a7..336c4b7a92 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 @@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag private[streaming] -class FlatMappedDStream[T: ClassManifest, U: ClassManifest]( +class FlatMappedDStream[T: ClassTag, U: ClassTag]( parent: DStream[T], flatMapFunc: T => Traversable[U] ) extends DStream[U](parent.ssc) { 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 e21bac4602..98b14cb224 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 @@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Duration, DStream, Job, Time} +import scala.reflect.ClassTag private[streaming] -class ForEachDStream[T: ClassManifest] ( +class ForEachDStream[T: ClassTag] ( parent: DStream[T], foreachFunc: (RDD[T], Time) => Unit ) extends DStream[Unit](parent.ssc) { 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 4294b07d91..23136f44fa 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 @@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag private[streaming] -class GlommedDStream[T: ClassManifest](parent: DStream[T]) +class GlommedDStream[T: ClassTag](parent: DStream[T]) extends DStream[Array[T]](parent.ssc) { override def dependencies = List(parent) 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 5329601a6f..8a04060e5b 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 @@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag private[streaming] -class MapPartitionedDStream[T: ClassManifest, U: ClassManifest]( +class MapPartitionedDStream[T: ClassTag, U: ClassTag]( parent: DStream[T], mapPartFunc: Iterator[T] => Iterator[U], preservePartitioning: Boolean 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 8290df90a2..0ce364fd46 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 @@ -20,9 +20,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ +import scala.reflect.ClassTag private[streaming] -class MapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest]( +class MapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag]( parent: DStream[(K, V)], mapValueFunc: V => U ) extends DStream[(K, U)](parent.ssc) { 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 b1682afea3..c0b7491d09 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 @@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag private[streaming] -class MappedDStream[T: ClassManifest, U: ClassManifest] ( +class MappedDStream[T: ClassTag, U: ClassTag] ( parent: DStream[T], mapFunc: T => U ) extends DStream[U](parent.ssc) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala index 15782f5c11..6f9477020a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala @@ -18,9 +18,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.StreamingContext +import scala.reflect.ClassTag private[streaming] -class PluggableInputDStream[T: ClassManifest]( +class PluggableInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, receiver: NetworkReceiver[T]) extends NetworkInputDStream[T](ssc_) { 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 7d9f3521b1..97325f8ea3 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 @@ -19,13 +19,13 @@ package org.apache.spark.streaming.dstream import org.apache.spark.rdd.RDD import org.apache.spark.rdd.UnionRDD - import scala.collection.mutable.Queue import scala.collection.mutable.ArrayBuffer import org.apache.spark.streaming.{Time, StreamingContext} +import scala.reflect.ClassTag private[streaming] -class QueueInputDStream[T: ClassManifest]( +class QueueInputDStream[T: ClassTag]( @transient ssc: StreamingContext, val queue: Queue[RDD[T]], oneAtATime: Boolean, 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 a95e66d761..e6e0022097 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 @@ -21,9 +21,10 @@ import org.apache.spark.Partitioner import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.streaming.{Duration, DStream, Time} +import scala.reflect.ClassTag private[streaming] -class ShuffledDStream[K: ClassManifest, V: ClassManifest, C: ClassManifest]( +class ShuffledDStream[K: ClassTag, V: ClassTag, C: ClassTag]( parent: DStream[(K,V)], createCombiner: V => C, mergeValue: (C, V) => C, 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 60485adef9..73e1ddf7a4 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 @@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Duration, DStream, Time} +import scala.reflect.ClassTag private[streaming] -class TransformedDStream[T: ClassManifest, U: ClassManifest] ( +class TransformedDStream[T: ClassTag, U: ClassTag] ( parent: DStream[T], transformFunc: (RDD[T], Time) => RDD[U] ) extends DStream[U](parent.ssc) { diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 783b8dea31..076fb53fa1 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -21,34 +21,36 @@ import com.google.common.base.Optional; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.io.Files; + import kafka.serializer.StringDecoder; + import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; + import scala.Tuple2; +import twitter4j.Status; + 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.dstream.SparkFlumeEvent; import org.apache.spark.streaming.JavaTestUtils; import org.apache.spark.streaming.JavaCheckpointTestUtils; -import org.apache.spark.streaming.InputStreamsSuite; import java.io.*; import java.util.*; import akka.actor.Props; import akka.zeromq.Subscribe; -import akka.util.ByteString; // The test suite itself is Serializable so that anonymous Function implementations can be @@ -85,8 +87,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(3L), Arrays.asList(1L)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream count = stream.count(); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream count = stream.count(); JavaTestUtils.attachTestOutputStream(count); List> result = JavaTestUtils.runStreams(ssc, 3, 3); assertOrderInvariantEquals(expected, result); @@ -102,8 +104,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(5,5), Arrays.asList(9,4)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream letterCount = stream.map(new Function() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream letterCount = stream.map(new Function() { @Override public Integer call(String s) throws Exception { return s.length(); @@ -128,8 +130,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(7,8,9,4,5,6), Arrays.asList(7,8,9)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream windowed = stream.window(new Duration(2000)); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream windowed = stream.window(new Duration(2000)); JavaTestUtils.attachTestOutputStream(windowed); List> result = JavaTestUtils.runStreams(ssc, 4, 4); @@ -152,8 +154,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(7,8,9,10,11,12,13,14,15,16,17,18), Arrays.asList(13,14,15,16,17,18)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream windowed = stream.window(new Duration(4000), new Duration(2000)); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream windowed = stream.window(new Duration(4000), new Duration(2000)); JavaTestUtils.attachTestOutputStream(windowed); List> result = JavaTestUtils.runStreams(ssc, 8, 4); @@ -170,8 +172,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList("giants"), Arrays.asList("yankees")); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream filtered = stream.filter(new Function() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream filtered = stream.filter(new Function() { @Override public Boolean call(String s) throws Exception { return s.contains("a"); @@ -193,8 +195,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(Arrays.asList("giants", "dodgers")), Arrays.asList(Arrays.asList("yankees", "red socks"))); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream glommed = stream.glom(); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream> glommed = stream.glom(); JavaTestUtils.attachTestOutputStream(glommed); List>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -211,8 +213,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList("GIANTSDODGERS"), Arrays.asList("YANKEESRED SOCKS")); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream mapped = stream.mapPartitions(new FlatMapFunction, String>() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream mapped = stream.mapPartitions(new FlatMapFunction, String>() { @Override public Iterable call(Iterator in) { String out = ""; @@ -254,8 +256,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(15), Arrays.asList(24)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream reduced = stream.reduce(new IntegerSum()); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream reduced = stream.reduce(new IntegerSum()); JavaTestUtils.attachTestOutputStream(reduced); List> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -275,8 +277,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(39), Arrays.asList(24)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream reducedWindowed = stream.reduceByWindow(new IntegerSum(), + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream reducedWindowed = stream.reduceByWindow(new IntegerSum(), new IntegerDifference(), new Duration(2000), new Duration(1000)); JavaTestUtils.attachTestOutputStream(reducedWindowed); List> result = JavaTestUtils.runStreams(ssc, 4, 4); @@ -349,8 +351,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList("b", "o", "o", "d","o","d","g","e","r","s"), Arrays.asList("a","t","h","l","e","t","i","c","s")); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream flatMapped = stream.flatMap(new FlatMapFunction() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream flatMapped = stream.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { return Lists.newArrayList(x.split("(?!^)")); @@ -396,8 +398,8 @@ public class JavaAPISuite implements Serializable { new Tuple2(9, "c"), new Tuple2(9, "s"))); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream flatMapped = stream.flatMap(new PairFlatMapFunction() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream flatMapped = stream.flatMap(new PairFlatMapFunction() { @Override public Iterable> call(String in) throws Exception { List> out = Lists.newArrayList(); @@ -430,10 +432,10 @@ public class JavaAPISuite implements Serializable { Arrays.asList(2,2,5,5), Arrays.asList(3,3,6,6)); - JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2); - JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2); + JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2); + JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2); - JavaDStream unioned = stream1.union(stream2); + JavaDStream unioned = stream1.union(stream2); JavaTestUtils.attachTestOutputStream(unioned); List> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -444,7 +446,7 @@ public class JavaAPISuite implements Serializable { * Performs an order-invariant comparison of lists representing two RDD streams. This allows * us to account for ordering variation within individual RDD's which occurs during windowing. */ - public static void assertOrderInvariantEquals( + public static > void assertOrderInvariantEquals( List> expected, List> actual) { for (List list: expected) { Collections.sort(list); @@ -467,11 +469,11 @@ public class JavaAPISuite implements Serializable { Arrays.asList(new Tuple2("giants", 6)), Arrays.asList(new Tuple2("yankees", 7))); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = stream.map( new PairFunction() { @Override - public Tuple2 call(String in) throws Exception { + public Tuple2 call(String in) throws Exception { return new Tuple2(in, in.length()); } }); @@ -1163,8 +1165,8 @@ public class JavaAPISuite implements Serializable { File tempDir = Files.createTempDir(); ssc.checkpoint(tempDir.getAbsolutePath()); - JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream letterCount = stream.map(new Function() { + JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream letterCount = stream.map(new Function() { @Override public Integer call(String s) throws Exception { return s.length(); @@ -1220,20 +1222,20 @@ public class JavaAPISuite implements Serializable { @Test public void testKafkaStream() { HashMap topics = Maps.newHashMap(); - JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics); - JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, + JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics); + JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK()); HashMap kafkaParams = Maps.newHashMap(); kafkaParams.put("zk.connect","localhost:12345"); kafkaParams.put("groupid","consumer-group"); - JavaDStream test3 = ssc.kafkaStream(String.class, StringDecoder.class, kafkaParams, topics, + JavaDStream test3 = ssc.kafkaStream(String.class, StringDecoder.class, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK()); } @Test public void testSocketTextStream() { - JavaDStream test = ssc.socketTextStream("localhost", 12345); + JavaDStream test = ssc.socketTextStream("localhost", 12345); } @Test @@ -1253,7 +1255,7 @@ public class JavaAPISuite implements Serializable { } } - JavaDStream test = ssc.socketStream( + JavaDStream test = ssc.socketStream( "localhost", 12345, new Converter(), @@ -1262,39 +1264,39 @@ public class JavaAPISuite implements Serializable { @Test public void testTextFileStream() { - JavaDStream test = ssc.textFileStream("/tmp/foo"); + JavaDStream test = ssc.textFileStream("/tmp/foo"); } @Test public void testRawSocketStream() { - JavaDStream test = ssc.rawSocketStream("localhost", 12345); + JavaDStream test = ssc.rawSocketStream("localhost", 12345); } @Test public void testFlumeStream() { - JavaDStream test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY()); + JavaDStream test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY()); } @Test public void testFileStream() { JavaPairDStream foo = - ssc.fileStream("/tmp/foo"); + ssc.>fileStream("/tmp/foo"); } @Test public void testTwitterStream() { String[] filters = new String[] { "good", "bad", "ugly" }; - JavaDStream test = ssc.twitterStream(filters, StorageLevel.MEMORY_ONLY()); + JavaDStream test = ssc.twitterStream(filters, StorageLevel.MEMORY_ONLY()); } @Test public void testActorStream() { - JavaDStream test = ssc.actorStream((Props)null, "TestActor", StorageLevel.MEMORY_ONLY()); + JavaDStream test = ssc.actorStream((Props)null, "TestActor", StorageLevel.MEMORY_ONLY()); } @Test public void testZeroMQStream() { - JavaDStream test = ssc.zeroMQStream("url", (Subscribe) null, new Function>() { + JavaDStream test = ssc.zeroMQStream("url", (Subscribe) null, new Function>() { @Override public Iterable call(byte[][] b) throws Exception { return null; -- cgit v1.2.3 From 34da58ae50cd97a1136d45484130addcf6ac8a33 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 10 Oct 2013 14:03:56 +0530 Subject: Changed message-frame-size to maximum-frame-size as property. Removed a test accidentally added during merge. --- .../src/main/scala/org/apache/spark/executor/Executor.scala | 2 +- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 4 ++-- core/src/test/scala/org/apache/spark/DistributedSuite.scala | 13 ------------- .../spark/scheduler/cluster/TaskResultGetterSuite.scala | 4 ++-- 4 files changed, 5 insertions(+), 18 deletions(-) (limited to 'core/src') 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 b4153f3533..3800063234 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -105,7 +105,7 @@ private[spark] class Executor( SparkEnv.set(env) env.metricsSystem.registerSource(executorSource) - private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size") + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size") // Start worker thread pool val threadPool = new ThreadPoolExecutor( diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index af1c36b34d..8daf50ab69 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -54,8 +54,8 @@ private[spark] object AkkaUtils { akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" akka.remote.netty.tcp.hostname = "%s" akka.remote.netty.tcp.port = %d - akka.remote.netty.tcp.connection-timeout = %ds - akka.remote.netty.tcp.message-frame-size = %d MiB + akka.remote.netty.tcp.connection-timeout = %d s + akka.remote.netty.tcp.maximum-frame-size = %dMiB akka.remote.netty.tcp.execution-pool-size = %d akka.actor.default-dispatcher.throughput = %d akka.remote.log-remote-lifecycle-events = %s diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 25b9c3eb78..988ab1747d 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -320,19 +320,6 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } } - test("job should fail if TaskResult exceeds Akka frame size") { - // We must use local-cluster mode since results are returned differently - // when running under LocalScheduler: - sc = new SparkContext("local-cluster[1,1,512]", "test") - val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size").toInt - val rdd = sc.parallelize(Seq(1)).map{x => new Array[Byte](akkaFrameSize)} - val exception = intercept[SparkException] { - rdd.reduce((x, y) => x) - } - exception.getMessage should endWith("result exceeded Akka frame size") - } - } object DistributedSuite { diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala index 370a3eb0eb..a00198db8c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala @@ -81,7 +81,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA test("handling results larger than Akka frame size") { val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size").toInt + sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) assert(result === 1.to(akkaFrameSize).toArray) @@ -102,7 +102,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA } scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size").toInt + sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) assert(result === 1.to(akkaFrameSize).toArray) -- cgit v1.2.3 From bfbd7e5d9f9e556961e79135dcde8f88fa31e2bc Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 10 Oct 2013 15:22:31 +0530 Subject: Fixed some scala warnings in core. --- core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 2 +- .../scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) (limited to 'core/src') 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 d7b45d4caa..c9c13f7a26 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 @@ -72,7 +72,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. */ - def mapPartitionsWithIndex[R: ClassManifest]( + def mapPartitionsWithIndex[R: ClassTag]( f: JFunction2[Int, java.util.Iterator[T], java.util.Iterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala index feec8ecfe4..b2a8f06472 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala @@ -87,7 +87,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterSche case cnf: ClassNotFoundException => val loader = Thread.currentThread.getContextClassLoader taskSetManager.abort("ClassNotFound with classloader: " + loader) - case ex => + case ex: Throwable => taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex)) } } @@ -111,7 +111,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterSche val loader = Thread.currentThread.getContextClassLoader logError( "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) - case ex => {} + case ex: Throwable => {} } scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) } -- cgit v1.2.3 From 52ccf4f859d92ed9e86d3720a983ac2c4a1c23bf Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Sat, 12 Oct 2013 14:34:14 +0800 Subject: deprecate "spark" script and SPAKR_CLASSPATH environment variable --- bin/spark | 92 ---------------------- core/pom.xml | 1 - .../main/scala/org/apache/spark/SparkContext.scala | 2 +- repl-bin/src/deb/bin/run | 3 +- repl/pom.xml | 1 - sbin/compute-classpath.cmd | 2 +- sbin/compute-classpath.sh | 2 +- 7 files changed, 4 insertions(+), 99 deletions(-) delete mode 100755 bin/spark (limited to 'core/src') diff --git a/bin/spark b/bin/spark deleted file mode 100755 index f5f7440d38..0000000000 --- a/bin/spark +++ /dev/null @@ -1,92 +0,0 @@ -#!/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: spark []" >&2 - echo "Usage: export SPARK_CLASSPATH before running the command" >&2 - exit 1 -fi - - -# Find the 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 - -# Set SPARK_MEM if it isn't already set -SPARK_MEM=${SPARK_MEM:-512m} -export SPARK_MEM - -# Set APP_MEM if it isn't already set, we use this for this process as the app driver process may need -# as much memory as specified in SPARK_MEM -APP_MEM=${APP_MEM:-512m} - -# Set JAVA_OPTS to be able to load native libraries and to set heap size -JAVA_OPTS="$OUR_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" -JAVA_OPTS="$JAVA_OPTS -Xms$APP_MEM -Xmx$APP_MEM" -# Load extra JAVA_OPTS from conf/java-opts, if it exists -if [ -e "$FWDIR/conf/java-opts" ] ; then - JAVA_OPTS="$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! - -if [ ! -f "$FWDIR/RELEASE" ]; then - # Exit if the user hasn't compiled Spark - 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 build Spark with sbt/sbt assembly before running this program" >&2 - exit 1 - fi -fi - -# Compute classpath using external script -CLASSPATH=`$FWDIR/sbin/compute-classpath.sh` -export CLASSPATH - -if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then - echo -n "Spark Command: " - echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" - echo "========================================" - echo -fi - -exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" diff --git a/core/pom.xml b/core/pom.xml index 9c2d6046a9..8359fefdb4 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -226,7 +226,6 @@ ${basedir}/.. 1 - ${spark.classpath} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 912ce752fb..ce7c4feaf6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -132,7 +132,7 @@ 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_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) { + for (key <- Seq("SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) { val value = System.getenv(key) if (value != null) { executorEnvs(key) = value diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run index 8b5d8300f2..d34f18906b 100755 --- a/repl-bin/src/deb/bin/run +++ b/repl-bin/src/deb/bin/run @@ -48,8 +48,7 @@ fi export JAVA_OPTS # Build up classpath -CLASSPATH="$SPARK_CLASSPATH" -CLASSPATH+=":$FWDIR/conf" +CLASSPATH=":$FWDIR/conf" for jar in `find $FWDIR -name '*jar'`; do CLASSPATH+=":$jar" done diff --git a/repl/pom.xml b/repl/pom.xml index 2826c0743c..f71184f865 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -125,7 +125,6 @@ ${basedir}/.. 1 - ${spark.classpath} diff --git a/sbin/compute-classpath.cmd b/sbin/compute-classpath.cmd index cf38188c4b..e0b8a8ef5f 100644 --- a/sbin/compute-classpath.cmd +++ b/sbin/compute-classpath.cmd @@ -29,7 +29,7 @@ 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 Build up classpath -set CLASSPATH=%SPARK_CLASSPATH%;%FWDIR%conf +set CLASSPATH=%FWDIR%conf if exist "%FWDIR%RELEASE" ( for %%d in ("%FWDIR%jars\spark-assembly*.jar") do ( set ASSEMBLY_JAR=%%d diff --git a/sbin/compute-classpath.sh b/sbin/compute-classpath.sh index d9217ecf77..cfe5fe7bef 100755 --- a/sbin/compute-classpath.sh +++ b/sbin/compute-classpath.sh @@ -31,7 +31,7 @@ if [ -e "$FWDIR/conf/spark-env.sh" ] ; then fi # Build up classpath -CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" +CLASSPATH="$FWDIR/conf" if [ -f "$FWDIR/RELEASE" ]; then ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar` else -- cgit v1.2.3 From 843727af99786a45cf29352b4e05df92c6b3b6b9 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Thu, 17 Oct 2013 22:17:06 -0700 Subject: Added a serializable wrapper for HyperLogLog --- .../spark/util/SerializableHyperLogLog.scala | 44 ++++++++++++++++++++++ 1 file changed, 44 insertions(+) create mode 100644 core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala new file mode 100644 index 0000000000..28a8accb33 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.util + +import java.io.{ObjectOutputStream, ObjectInputStream} +import com.clearspring.analytics.stream.cardinality.{ICardinality, HyperLogLog} + +/** + * A wrapper around com.clearspring.analytics.stream.cardinality.HyperLogLog that is serializable. + */ +private[spark] +class SerializableHyperLogLog(@transient var value: ICardinality) extends Serializable { + + + def merge(other: SerializableHyperLogLog) = new SerializableHyperLogLog(value.merge(other.value)) + + private def readObject(in: ObjectInputStream) { + val byteLength = in.readInt() + val bytes = new Array[Byte](byteLength) + in.readFully(bytes) + value = HyperLogLog.Builder.build(bytes) + } + + private def writeObject(out: ObjectOutputStream) { + val bytes = value.getBytes() + out.writeInt(bytes.length) + out.write(bytes) + } +} -- cgit v1.2.3 From 1a701358c0811c7f270132291e0646fd806e4984 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Thu, 17 Oct 2013 22:24:48 -0700 Subject: Added a countDistinct method to RDD that takes takes an accuracy parameter and returns the (approximate) number of distinct elements in the RDD. --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 26 +++++++++++++++++++++- .../test/scala/org/apache/spark/rdd/RDDSuite.scala | 13 +++++++++++ 2 files changed, 38 insertions(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 0355618e43..09932db5ea 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -30,6 +30,7 @@ import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.TextOutputFormat import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} +import com.clearspring.analytics.stream.cardinality.HyperLogLog import org.apache.spark.Partitioner._ import org.apache.spark.api.java.JavaRDD @@ -38,7 +39,7 @@ import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{Utils, BoundedPriorityQueue} +import org.apache.spark.util.{Utils, BoundedPriorityQueue, SerializableHyperLogLog} import org.apache.spark.SparkContext._ import org.apache.spark._ @@ -765,6 +766,29 @@ abstract class RDD[T: ClassManifest]( sc.runApproximateJob(this, countPartition, evaluator, timeout) } + /** + * Return approximate number of distinct elements in the RDD. + * + * The accuracy of approximation can be controlled through the relative standard diviation + * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in + * more accurate counts but increase the memory footprint and vise versa. The default value of + * relativeSD is 0.05. + */ + def countDistinct(relativeSD: Double = 0.05): Long = { + + def hllCountPartition(iter: Iterator[T]): Iterator[SerializableHyperLogLog] = { + val hllCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) + while (iter.hasNext) { + val v = iter.next() + hllCounter.value.offer(v) + } + Iterator(hllCounter) + } + def mergeCounters(c1: SerializableHyperLogLog, c2: SerializableHyperLogLog): SerializableHyperLogLog = c1.merge(c2) + + mapPartitions(hllCountPartition).reduce(mergeCounters).value.cardinality() + } + /** * Take the first num elements of the RDD. It works by first scanning one partition, and use the * results from that partition to estimate the number of additional partitions needed to satisfy diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 6d1bc5e296..6baf9c7ece 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -63,6 +63,19 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } + test("Approximate distinct count") { + + def error(est: Long, size: Long) = math.abs(est - size)/size.toDouble + + val size = 100 + val uniformDistro = for (i <- 1 to 100000) yield i % size + val simpleRdd = sc.makeRDD(uniformDistro) + assert( error(simpleRdd.countDistinct(0.2), size) < 0.2) + assert( error(simpleRdd.countDistinct(0.05), size) < 0.05) + assert( error(simpleRdd.countDistinct(0.01), size) < 0.01) + assert( error(simpleRdd.countDistinct(0.001), size) < 0.001) + } + test("SparkContext.union") { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) assert(sc.union(nums).collect().toList === List(1, 2, 3, 4)) -- cgit v1.2.3 From ec5df800fdb0109314c0d5cd6dcac2ecbb9433d6 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Thu, 17 Oct 2013 22:26:00 -0700 Subject: Added countDistinctByKey to PairRDDFunctions that counts the approximate number of unique values for each key in the RDD. --- .../org/apache/spark/rdd/PairRDDFunctions.scala | 51 ++++++++++++++++++++++ .../apache/spark/rdd/PairRDDFunctionsSuite.scala | 30 +++++++++++++ 2 files changed, 81 insertions(+) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 93b78e1232..f34593f0b6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -39,12 +39,15 @@ import org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob} import org.apache.hadoop.mapreduce.{RecordWriter => NewRecordWriter} +import com.clearspring.analytics.stream.cardinality.HyperLogLog + import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.Aggregator import org.apache.spark.Partitioner import org.apache.spark.Partitioner.defaultPartitioner +import org.apache.spark.util.SerializableHyperLogLog /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. @@ -206,6 +209,54 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) self.map(_._1).countByValueApprox(timeout, confidence) } + /** + * Return approximate number of distinct values for each key in this RDD. + * The accuracy of approximation can be controlled through the relative standard diviation + * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in + * more accurate counts but increase the memory footprint and vise versa. Uses the provided + * Partitioner to partition the output RDD. + */ + def countDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = { + val createHLL = (v: V) => { + val hll = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) + val bres = hll.value.offer(v) + hll + } + val mergeValueHLL = (hll: SerializableHyperLogLog, v: V) => { + hll.value.offer(v) + hll + } + val mergeHLL = (h1: SerializableHyperLogLog, h2: SerializableHyperLogLog) => h1.merge(h2) + + combineByKey(createHLL, mergeValueHLL, mergeHLL, partitioner).map { + case (k, v) => (k, v.value.cardinality()) + } + } + + /** + * Return approximate number of distinct values for each key in this RDD. + * The accuracy of approximation can be controlled through the relative standard diviation + * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in + * more accurate counts but increase the memory footprint and vise versa. HashPartitions the + * output RDD into numPartitions. + * + */ + def countDistinctByKey(relativeSD: Double, numPartitions: Int): RDD[(K, Long)] = { + countDistinctByKey(relativeSD, new HashPartitioner(numPartitions)) + } + + /** + * Return approximate number of distinct values for each key this RDD. + * The accuracy of approximation can be controlled through the relative standard diviation + * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in + * more accurate counts but increase the memory footprint and vise versa. The default value of + * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism + * level. + */ + def countDistinctByKey(relativeSD: Double = 0.05): RDD[(K, Long)] = { + countDistinctByKey(relativeSD, defaultPartitioner(self)) + } + /** * 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 diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 57d3382ed0..d81bc8cb4c 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -109,6 +109,36 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { assert(deps.size === 2) // ShuffledRDD, ParallelCollection. } + test("countDistinctByKey") { + def error(est: Long, size: Long) = math.abs(est - size)/size.toDouble + + /* Since HyperLogLog unique counting is approximate, and the relative standard deviation is + only a statistical bound, the tests can fail for large values of relativeSD. We will be using + relatively tight error bounds to check correctness of functionality rather than checking + whether the approximation conforms with the requested bound. + */ + val relativeSD = 0.001 + + val stacked = (1 to 100).flatMap(i => (1 to i).map(j => (i, j))) + val rdd1 = sc.parallelize(stacked) + val counted1 = rdd1.countDistinctByKey(relativeSD).collect() + counted1.foreach{ + case(k, count) => assert(math.abs(error(count, k)) < relativeSD) + } + + import scala.util.Random + val rnd = new Random() + val randStacked = (1 to 100).flatMap{i => + val num = rnd.nextInt%500 + (1 to num).map(j => (num, j)) + } + val rdd2 = sc.parallelize(randStacked) + val counted2 = rdd2.countDistinctByKey(relativeSD, 4).collect() + counted2.foreach{ + case(k, count) => assert(math.abs(error(count, k)) < relativeSD) + } + } + 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'))) -- cgit v1.2.3 From b611d9a65c0eda8ca7ceb015773ea4a4e26f2640 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Thu, 17 Oct 2013 23:05:22 -0700 Subject: Fixed document typo --- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 6 +++--- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index f34593f0b6..d778692f45 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -211,7 +211,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) /** * Return approximate number of distinct values for each key in this RDD. - * The accuracy of approximation can be controlled through the relative standard diviation + * The accuracy of approximation can be controlled through the relative standard deviation * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in * more accurate counts but increase the memory footprint and vise versa. Uses the provided * Partitioner to partition the output RDD. @@ -235,7 +235,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) /** * Return approximate number of distinct values for each key in this RDD. - * The accuracy of approximation can be controlled through the relative standard diviation + * The accuracy of approximation can be controlled through the relative standard deviation * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in * more accurate counts but increase the memory footprint and vise versa. HashPartitions the * output RDD into numPartitions. @@ -247,7 +247,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) /** * Return approximate number of distinct values for each key this RDD. - * The accuracy of approximation can be controlled through the relative standard diviation + * The accuracy of approximation can be controlled through the relative standard deviation * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in * more accurate counts but increase the memory footprint and vise versa. The default value of * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 09932db5ea..38fa96fd6d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -769,7 +769,7 @@ abstract class RDD[T: ClassManifest]( /** * Return approximate number of distinct elements in the RDD. * - * The accuracy of approximation can be controlled through the relative standard diviation + * The accuracy of approximation can be controlled through the relative standard deviation * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in * more accurate counts but increase the memory footprint and vise versa. The default value of * relativeSD is 0.05. -- cgit v1.2.3 From 79868fe7246d8e6d57e0a376b2593fabea9a9d83 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Thu, 17 Oct 2013 23:39:20 -0700 Subject: Improved code style. --- .../scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- .../org/apache/spark/rdd/PairRDDFunctionsSuite.scala | 18 +++++++++++------- .../src/test/scala/org/apache/spark/rdd/RDDSuite.scala | 12 ++++++------ 4 files changed, 19 insertions(+), 15 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index d778692f45..322b519bd2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -219,7 +219,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) def countDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = { val createHLL = (v: V) => { val hll = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) - val bres = hll.value.offer(v) + hll.value.offer(v) hll } val mergeValueHLL = (hll: SerializableHyperLogLog, v: V) => { diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 38fa96fd6d..e23e7a63a1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -784,7 +784,7 @@ abstract class RDD[T: ClassManifest]( } Iterator(hllCounter) } - def mergeCounters(c1: SerializableHyperLogLog, c2: SerializableHyperLogLog): SerializableHyperLogLog = c1.merge(c2) + def mergeCounters(c1: SerializableHyperLogLog, c2: SerializableHyperLogLog) = c1.merge(c2) mapPartitions(hllCountPartition).reduce(mergeCounters).value.cardinality() } diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index d81bc8cb4c..5683ada78a 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.rdd import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet +import scala.util.Random import org.scalatest.FunSuite @@ -110,15 +111,17 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { } test("countDistinctByKey") { - def error(est: Long, size: Long) = math.abs(est - size)/size.toDouble + def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble /* Since HyperLogLog unique counting is approximate, and the relative standard deviation is - only a statistical bound, the tests can fail for large values of relativeSD. We will be using - relatively tight error bounds to check correctness of functionality rather than checking - whether the approximation conforms with the requested bound. + * only a statistical bound, the tests can fail for large values of relativeSD. We will be using + * relatively tight error bounds to check correctness of functionality rather than checking + * whether the approximation conforms with the requested bound. */ val relativeSD = 0.001 + // For each value i, there are i tuples with first element equal to i. + // Therefore, the expected count for key i would be i. val stacked = (1 to 100).flatMap(i => (1 to i).map(j => (i, j))) val rdd1 = sc.parallelize(stacked) val counted1 = rdd1.countDistinctByKey(relativeSD).collect() @@ -126,10 +129,11 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { case(k, count) => assert(math.abs(error(count, k)) < relativeSD) } - import scala.util.Random val rnd = new Random() - val randStacked = (1 to 100).flatMap{i => - val num = rnd.nextInt%500 + + // The expected count for key num would be num + val randStacked = (1 to 100).flatMap { i => + val num = rnd.nextInt % 500 (1 to num).map(j => (num, j)) } val rdd2 = sc.parallelize(randStacked) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 6baf9c7ece..413ea85322 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -63,17 +63,17 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } - test("Approximate distinct count") { + test("countDistinct") { - def error(est: Long, size: Long) = math.abs(est - size)/size.toDouble + def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble val size = 100 val uniformDistro = for (i <- 1 to 100000) yield i % size val simpleRdd = sc.makeRDD(uniformDistro) - assert( error(simpleRdd.countDistinct(0.2), size) < 0.2) - assert( error(simpleRdd.countDistinct(0.05), size) < 0.05) - assert( error(simpleRdd.countDistinct(0.01), size) < 0.01) - assert( error(simpleRdd.countDistinct(0.001), size) < 0.001) + assert(error(simpleRdd.countDistinct(0.2), size) < 0.2) + assert(error(simpleRdd.countDistinct(0.05), size) < 0.05) + assert(error(simpleRdd.countDistinct(0.01), size) < 0.01) + assert(error(simpleRdd.countDistinct(0.001), size) < 0.001) } test("SparkContext.union") { -- cgit v1.2.3 From 2d511ab320a85eccafbb9e51a2183b07114bbaa1 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Fri, 18 Oct 2013 15:30:45 -0700 Subject: Made SerializableHyperLogLog Externalizable and added Kryo tests --- .../scala/org/apache/spark/util/SerializableHyperLogLog.scala | 11 ++++++----- .../org/apache/spark/serializer/KryoSerializerSuite.scala | 4 ++++ 2 files changed, 10 insertions(+), 5 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala index 28a8accb33..9cfd41407f 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala @@ -17,26 +17,27 @@ package org.apache.spark.util -import java.io.{ObjectOutputStream, ObjectInputStream} +import java.io.{Externalizable, ObjectOutput, ObjectInput} import com.clearspring.analytics.stream.cardinality.{ICardinality, HyperLogLog} /** - * A wrapper around com.clearspring.analytics.stream.cardinality.HyperLogLog that is serializable. + * A wrapper around [[com.clearspring.analytics.stream.cardinality.HyperLogLog]] that is serializable. */ private[spark] -class SerializableHyperLogLog(@transient var value: ICardinality) extends Serializable { +class SerializableHyperLogLog(var value: ICardinality) extends Externalizable { + def this() = this(null) // For deserialization def merge(other: SerializableHyperLogLog) = new SerializableHyperLogLog(value.merge(other.value)) - private def readObject(in: ObjectInputStream) { + def readExternal(in: ObjectInput) { val byteLength = in.readInt() val bytes = new Array[Byte](byteLength) in.readFully(bytes) value = HyperLogLog.Builder.build(bytes) } - private def writeObject(out: ObjectOutputStream) { + def writeExternal(out: ObjectOutput) { val bytes = value.getBytes() out.writeInt(bytes.length) out.write(bytes) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index c016c51171..18529710fe 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -172,6 +172,10 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { assert (sc.parallelize( Array((1, 11), (2, 22), (3, 33)) ).collect().head === (1, 11)) } + test("kryo with SerializableHyperLogLog") { + assert(sc.parallelize( Array(1, 2, 3, 2, 3, 3, 2, 3, 1) ).countDistinct(0.01) === 3) + } + test("kryo with reduce") { val control = 1 :: 2 :: Nil val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)) -- cgit v1.2.3 From 2a37235825cecd3f75286d11456c6e3cb13d4327 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 18 Oct 2013 00:07:49 -0700 Subject: Initial commit of adding histogram functionality to the DoubleRDDFunctions. --- .../org/apache/spark/api/java/JavaDoubleRDD.scala | 32 +++ .../org/apache/spark/rdd/DoubleRDDFunctions.scala | 134 ++++++++++++ .../org/apache/spark/rdd/DoubleRDDSuite.scala | 233 +++++++++++++++++++++ 3 files changed, 399 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala (limited to 'core/src') 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 5fd1fab580..d2a2818e59 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 @@ -26,6 +26,8 @@ import org.apache.spark.storage.StorageLevel import java.lang.Double import org.apache.spark.Partitioner +import scala.collection.JavaConverters._ + class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, JavaDoubleRDD] { override val classManifest: ClassManifest[Double] = implicitly[ClassManifest[Double]] @@ -158,6 +160,36 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav /** (Experimental) Approximate operation to return the sum within a timeout. */ def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout) + + /** + * Compute a histogram of the data using bucketCount number of buckets evenly + * spaced between the minimum and maximum of the RDD. For example if the min + * value is 0 and the max is 100 and there are two buckets the resulting + * buckets will be [0,50) [50,100]. bucketCount must be at least 1 + * If the RDD contains infinity, NaN throws an exception + * If the elements in RDD do not vary (max == min) throws an exception + */ + def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = { + val result = srdd.histogram(bucketCount) + (result._1.map(scala.Double.box(_)), result._2) + } + /** + * Compute a histogram using the provided buckets. The buckets are all open + * to the left except for the last which is closed + * e.g. for the array + * [1,10,20,50] the buckets are [1,10) [10,20) [20,50] + * e.g 1<=x<10 , 10<=x<20, 20<=x<50 + * And on the input of 1 and 50 we would have a histogram of 1,0,0 + * + * Note: if your histogram is evenly spaced (e.g. [0,10,20,30]) this switches + * from an O(log n) inseration to O(1) per element. (where n = # buckets) + * buckets must be sorted and not contain any duplicates. + * buckets array must be at least two elements + * All NaN entries are treated the same. + */ + def histogram(buckets: Array[Double]): Array[Long] = { + srdd.histogram(buckets.map(_.toDouble)) + } } object JavaDoubleRDD { diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index a4bec41752..776a83cefe 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -24,6 +24,8 @@ import org.apache.spark.partial.SumEvaluator import org.apache.spark.util.StatCounter import org.apache.spark.{TaskContext, Logging} +import scala.collection.immutable.NumericRange + /** * Extra functions available on RDDs of Doubles through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. @@ -76,4 +78,136 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { val evaluator = new SumEvaluator(self.partitions.size, confidence) self.context.runApproximateJob(self, processPartition, evaluator, timeout) } + + /** + * Compute a histogram of the data using bucketCount number of buckets evenly + * spaced between the minimum and maximum of the RDD. For example if the min + * value is 0 and the max is 100 and there are two buckets the resulting + * buckets will be [0,50) [50,100]. bucketCount must be at least 1 + * If the RDD contains infinity, NaN throws an exception + * If the elements in RDD do not vary (max == min) throws an exception + */ + def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = { + // Compute the minimum and the maxium + val (max: Double, min: Double) = self.mapPartitions { items => + Iterator(items.foldRight(-1/0.0, Double.NaN)((e: Double, x: Pair[Double, Double]) => + (x._1.max(e),x._2.min(e)))) + }.reduce { (maxmin1, maxmin2) => + (maxmin1._1.max(maxmin2._1), maxmin1._2.min(maxmin2._2)) + } + if (max.isNaN() || max.isInfinity || min.isInfinity ) { + throw new UnsupportedOperationException("Histogram on either an empty RDD or RDD containing +-infinity or NaN") + } + if (max == min) { + throw new UnsupportedOperationException("Histogram with no range in elements") + } + val increment: Double = (max-min)/bucketCount.toDouble + val range = Range.Double.inclusive(min, max, increment) + val buckets: Array[Double] = range.toArray + (buckets,histogram(buckets)) + } + /** + * Compute a histogram using the provided buckets. The buckets are all open + * to the left except for the last which is closed + * e.g. for the array + * [1,10,20,50] the buckets are [1,10) [10,20) [20,50] + * e.g 1<=x<10 , 10<=x<20, 20<=x<50 + * And on the input of 1 and 50 we would have a histogram of 1,0,0 + * + * Note: if your histogram is evenly spaced (e.g. [0,10,20,30]) this switches + * from an O(log n) inseration to O(1) per element. (where n = # buckets) + * buckets must be sorted and not contain any duplicates. + * buckets array must be at least two elements + * All NaN entries are treated the same. + */ + def histogram(buckets: Array[Double]): Array[Long] = { + if (buckets.length < 2) { + throw new IllegalArgumentException("buckets array must have at least two elements") + } + // The histogramPartition function computes the partail histogram for a given + // partition. The provided bucketFunction determines which bucket in the array + // to increment or returns None if there is no bucket. This is done so we can + // specialize for uniformly distributed buckets and save the O(log n) binary + // search cost. + def histogramPartition(bucketFunction: (Double) => Option[Int])(iter: Iterator[Double]): Iterator[Array[Long]] = { + val counters = new Array[Long](buckets.length-1) + while (iter.hasNext) { + bucketFunction(iter.next()) match { + case Some(x: Int) => {counters(x)+=1} + case _ => {} + } + } + Iterator(counters) + } + // Merge the counters. + def mergeCounters(a1: Array[Long], a2: Array[Long]): Array[Long] = { + a1.indices.foreach(i => a1(i) += a2(i)) + a1 + } + // Basic bucket function. This works using Java's built in Array + // binary search. Takes log(size(buckets)) + def basicBucketFunction(e: Double): Option[Int] = { + val location = java.util.Arrays.binarySearch(buckets, e) + if (location < 0) { + // If the location is less than 0 then the insertion point in the array + // to keep it sorted is -location-1 + val insertionPoint = -location-1 + // If we have to insert before the first element or after the last one + // its out of bounds. + // We do this rather than buckets.lengthCompare(insertionPoint) + // because Array[Double] fails to override it (for now). + if (insertionPoint > 0 && insertionPoint < buckets.length) { + Some(insertionPoint-1) + } else { + None + } + } else if (location < buckets.length-1) { + // Exact match, just insert here + Some(location) + } else { + // Exact match to the last element + Some(location-1) + } + } + // Determine the bucket function in constant time. Requires that buckets are evenly spaced + def fastBucketFunction(min: Double, increment: Double, count: Int)(e: Double): Option[Int] = { + // If our input is not a number unless the increment is also NaN then we fail fast + if (e.isNaN()) { + return None + } + val bucketNumber = (e-min)/(increment) + // We do this rather than buckets.lengthCompare(bucketNumber) + // because Array[Double] fails to override it (for now). + if (bucketNumber > count || bucketNumber < 0) { + None + } else { + Some(bucketNumber.toInt.min(count-1)) + } + } + def evenlySpaced(buckets: Array[Double]): Boolean = { + val delta = buckets(1)-buckets(0) + // Technically you could have an evenly spaced bucket with NaN + // increments but then its a single bucket and this makes the + // fastBucketFunction simpler. + if (delta.isNaN() || delta.isInfinite()) { + return false + } + for (i <- 1 to buckets.length-1) { + if (buckets(i)-buckets(i-1) != delta) { + return false + } + } + true + } + // Decide which bucket function to pass to histogramPartition. We decide here + // rather than having a general function so that the decission need only be made + // once rather than once per shard + val bucketFunction = if (evenlySpaced(buckets)) { + fastBucketFunction(buckets(0), buckets(1)-buckets(0), buckets.length-1) _ + } else { + basicBucketFunction _ + } + self.mapPartitions(histogramPartition(bucketFunction)).reduce(mergeCounters) + } + } diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala new file mode 100644 index 0000000000..2ec7173511 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.rdd + +import scala.math.abs +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd._ +import org.apache.spark._ + +class DoubleRDDSuite extends FunSuite with SharedSparkContext { + // Verify tests on the histogram functionality. We test with both evenly + // and non-evenly spaced buckets as the bucket lookup function changes. + test("WorksOnEmpty") { + // Make sure that it works on an empty input + val rdd: RDD[Double] = sc.parallelize(Seq()) + val buckets: Array[Double] = Array(0.0, 10.0) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(0) + assert(histogramResults === expectedHistogramResults) + } + test("WorksWithOutOfRangeWithOneBucket") { + // Verify that if all of the elements are out of range the counts are zero + val rdd: RDD[Double] = sc.parallelize(Seq(10.01,-0.01)) + val buckets: Array[Double] = Array(0.0, 10.0) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(0) + assert(histogramResults === expectedHistogramResults) + } + test("WorksInRangeWithOneBucket") { + // Verify the basic case of one bucket and all elements in that bucket works + val rdd: RDD[Double] = sc.parallelize(Seq(1,2,3,4)) + val buckets: Array[Double] = Array(0.0, 10.0) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(4) + assert(histogramResults === expectedHistogramResults) + } + test("WorksInRangeWithOneBucketExactMatch") { + // Verify the basic case of one bucket and all elements in that bucket works + val rdd: RDD[Double] = sc.parallelize(Seq(1,2,3,4)) + val buckets: Array[Double] = Array(1.0, 4.0) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(4) + assert(histogramResults === expectedHistogramResults) + } + test("WorksWithOutOfRangeWithTwoBuckets") { + // Verify that out of range works with two buckets + val rdd: RDD[Double] = sc.parallelize(Seq(10.01,-0.01)) + val buckets: Array[Double] = Array(0.0, 5.0, 10.0) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(0,0) + assert(histogramResults === expectedHistogramResults) + } + test("WorksWithOutOfRangeWithTwoUnEvenBuckets") { + // Verify that out of range works with two un even buckets + val rdd: RDD[Double] = sc.parallelize(Seq(10.01,-0.01)) + val buckets: Array[Double] = Array(0.0, 4.0, 10.0) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(0,0) + assert(histogramResults === expectedHistogramResults) + } + test("WorksInRangeWithTwoBuckets") { + // Make sure that it works with two equally spaced buckets and elements in each + val rdd: RDD[Double] = sc.parallelize(Seq(1,2,3,5,6)) + val buckets: Array[Double] = Array(0.0, 5.0, 10.0) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(3,2) + assert(histogramResults === expectedHistogramResults) + } + test("WorksInRangeWithTwoBucketsAndNaN") { + // Make sure that it works with two equally spaced buckets and elements in each + val rdd: RDD[Double] = sc.parallelize(Seq(1,2,3,5,6,Double.NaN)) + val buckets: Array[Double] = Array(0.0, 5.0, 10.0) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(3,2) + assert(histogramResults === expectedHistogramResults) + } + test("WorksInRangeWithTwoUnevenBuckets") { + // Make sure that it works with two unequally spaced buckets and elements in each + val rdd: RDD[Double] = sc.parallelize(Seq(1,2,3,5,6)) + val buckets: Array[Double] = Array(0.0, 5.0, 11.0) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(3,2) + assert(histogramResults === expectedHistogramResults) + } + test("WorksMixedRangeWithTwoUnevenBuckets") { + // Make sure that it works with two unequally spaced buckets and elements in each + val rdd: RDD[Double] = sc.parallelize(Seq(-0.01,0.0,1,2,3,5,6,11.0,11.01)) + val buckets: Array[Double] = Array(0.0, 5.0, 11.0) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(4,3) + assert(histogramResults === expectedHistogramResults) + } + test("WorksMixedRangeWithFourUnevenBuckets") { + // Make sure that it works with two unequally spaced buckets and elements in each + val rdd: RDD[Double] = sc.parallelize(Seq(-0.01,0.0,1,2,3,5,6,11.01,12.0,199.0,200.0,200.1)) + val buckets: Array[Double] = Array(0.0, 5.0, 11.0,12.0,200.0) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(4,2,1,3) + assert(histogramResults === expectedHistogramResults) + } + test("WorksMixedRangeWithUnevenBucketsAndNaN") { + // Make sure that it works with two unequally spaced buckets and elements in each + val rdd: RDD[Double] = sc.parallelize(Seq(-0.01,0.0,1,2,3,5,6,11.01,12.0,199.0,200.0,200.1,Double.NaN)) + val buckets: Array[Double] = Array(0.0, 5.0, 11.0,12.0,200.0) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(4,2,1,3) + assert(histogramResults === expectedHistogramResults) + } + // Make sure this works with a NaN end bucket + test("WorksMixedRangeWithUnevenBucketsAndNaNAndNaNRange") { + // Make sure that it works with two unequally spaced buckets and elements in each + val rdd: RDD[Double] = sc.parallelize(Seq(-0.01,0.0,1,2,3,5,6,11.01,12.0,199.0,200.0,200.1,Double.NaN)) + val buckets: Array[Double] = Array(0.0, 5.0, 11.0,12.0,200.0,Double.NaN) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(4,2,1,2,3) + assert(histogramResults === expectedHistogramResults) + } + // Make sure this works with a NaN end bucket and an inifity + test("WorksMixedRangeWithUnevenBucketsAndNaNAndNaNRangeAndInfity") { + // Make sure that it works with two unequally spaced buckets and elements in each + val rdd: RDD[Double] = sc.parallelize(Seq(-0.01,0.0,1,2,3,5,6,11.01,12.0,199.0,200.0,200.1,1.0/0.0,-1.0/0.0,Double.NaN)) + val buckets: Array[Double] = Array(0.0, 5.0, 11.0,12.0,200.0,Double.NaN) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(4,2,1,2,4) + assert(histogramResults === expectedHistogramResults) + } + test("WorksWithOutOfRangeWithInfiniteBuckets") { + // Verify that out of range works with two buckets + val rdd: RDD[Double] = sc.parallelize(Seq(10.01,-0.01,Double.NaN)) + val buckets: Array[Double] = Array(-1.0/0.0 ,0.0, 1.0/0.0) + val histogramResults: Array[Long] = rdd.histogram(buckets) + val expectedHistogramResults: Array[Long] = Array(1,1) + assert(histogramResults === expectedHistogramResults) + } + // Test the failure mode with an invalid bucket array + test("ThrowsExceptionOnInvalidBucketArray") { + val rdd: RDD[Double] = sc.parallelize(Seq(1.0)) + // Empty array + intercept[IllegalArgumentException]{ + val buckets: Array[Double] = Array.empty[Double] + val result = rdd.histogram(buckets) + } + // Single element array + intercept[IllegalArgumentException] + { + val buckets: Array[Double] = Array(1.0) + val result = rdd.histogram(buckets) + } + } + + // Test automatic histogram function + test("WorksWithoutBucketsBasic") { + // Verify the basic case of one bucket and all elements in that bucket works + val rdd: RDD[Double] = sc.parallelize(Seq(1,2,3,4)) + val (histogramBuckets, histogramResults) = rdd.histogram(1) + val expectedHistogramResults: Array[Long] = Array(4) + val expectedHistogramBuckets: Array[Double] = Array(1.0,4.0) + assert(histogramResults === expectedHistogramResults) + assert(histogramBuckets === expectedHistogramBuckets) + } + test("WorksWithoutBucketsBasicTwo") { + // Verify the basic case of one bucket and all elements in that bucket works + val rdd: RDD[Double] = sc.parallelize(Seq(1,2,3,4)) + val (histogramBuckets, histogramResults) = rdd.histogram(2) + val expectedHistogramResults: Array[Long] = Array(2,2) + val expectedHistogramBuckets: Array[Double] = Array(1.0,2.5,4.0) + assert(histogramResults === expectedHistogramResults) + assert(histogramBuckets === expectedHistogramBuckets) + } + test("WorksWithoutBucketsWithMoreRequestedThanElements") { + // Verify the basic case of one bucket and all elements in that bucket works + val rdd: RDD[Double] = sc.parallelize(Seq(1,2)) + val (histogramBuckets, histogramResults) = rdd.histogram(10) + val expectedHistogramResults: Array[Long] = + Array(1, 0, 0, 0, 0, 0, 0, 0, 0, 1) + val expectedHistogramBuckets: Array[Double] = + Array(1.0, 1.1, 1.2, 1.3, 1.4, 1.5, 1.6, 1.7, 1.8, 1.9, 2.0) + assert(histogramResults === expectedHistogramResults) + assert(histogramBuckets === expectedHistogramBuckets) + } + // Test the failure mode with an invalid RDD + test("ThrowsExceptionOnInvalidRDDs") { + // infinity + intercept[UnsupportedOperationException]{ + val rdd: RDD[Double] = sc.parallelize(Seq(1,1.0/0.0)) + val result = rdd.histogram(1) + } + // NaN + intercept[UnsupportedOperationException] + { + val rdd: RDD[Double] = sc.parallelize(Seq(1,Double.NaN)) + val result = rdd.histogram(1) + } + // Empty + intercept[UnsupportedOperationException] + { + val rdd: RDD[Double] = sc.parallelize(Seq()) + val result = rdd.histogram(1) + } + // Single element + intercept[UnsupportedOperationException] + { + val rdd: RDD[Double] = sc.parallelize(Seq(1)) + val result = rdd.histogram(1) + } + // No Range + intercept[UnsupportedOperationException] + { + val rdd: RDD[Double] = sc.parallelize(Seq(1,1,1)) + val result = rdd.histogram(1) + } + } + +} -- cgit v1.2.3 From e58c69d955ef8faacb794a0c1666b21c1606453e Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sun, 20 Oct 2013 01:17:13 -0700 Subject: Add tests for the Java implementation. --- core/src/test/scala/org/apache/spark/JavaAPISuite.java | 14 ++++++++++++++ 1 file changed, 14 insertions(+) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java index 591c1d498d..8a9c6e63e0 100644 --- a/core/src/test/scala/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java @@ -364,6 +364,20 @@ public class JavaAPISuite implements Serializable { List take = rdd.take(5); } + @Test + public void javaDoubleRDDHistoGram() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + // Test using generated buckets + Tuple2 results = rdd.histogram(2); + Double[] expected_buckets = {1.0, 2.5, 4.0}; + long[] expected_counts = {2, 2}; + Assert.assertArrayEquals(expected_buckets, results._1); + Assert.assertArrayEquals(expected_counts, results._2); + // Test with provided buckets + long[] histogram = rdd.histogram(expected_buckets); + Assert.assertArrayEquals(expected_counts, histogram); + } + @Test public void map() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); -- cgit v1.2.3 From 699f7d28c0347cb516fa17f94b53d7bc50f18346 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 21 Oct 2013 00:10:03 -0700 Subject: CR feedback --- .../org/apache/spark/api/java/JavaDoubleRDD.scala | 18 ++- .../org/apache/spark/rdd/DoubleRDDFunctions.scala | 68 +++++----- .../org/apache/spark/rdd/DoubleRDDSuite.scala | 140 ++++++++++++--------- 3 files changed, 125 insertions(+), 101 deletions(-) (limited to 'core/src') 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 d2a2818e59..b002468442 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 @@ -167,12 +167,13 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav * value is 0 and the max is 100 and there are two buckets the resulting * buckets will be [0,50) [50,100]. bucketCount must be at least 1 * If the RDD contains infinity, NaN throws an exception - * If the elements in RDD do not vary (max == min) throws an exception + * If the elements in RDD do not vary (max == min) always returns a single bucket. */ def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = { val result = srdd.histogram(bucketCount) (result._1.map(scala.Double.box(_)), result._2) } + /** * Compute a histogram using the provided buckets. The buckets are all open * to the left except for the last which is closed @@ -181,14 +182,21 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav * e.g 1<=x<10 , 10<=x<20, 20<=x<50 * And on the input of 1 and 50 we would have a histogram of 1,0,0 * - * Note: if your histogram is evenly spaced (e.g. [0,10,20,30]) this switches - * from an O(log n) inseration to O(1) per element. (where n = # buckets) + * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched + * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets + * to true. * buckets must be sorted and not contain any duplicates. * buckets array must be at least two elements - * All NaN entries are treated the same. + * All NaN entries are treated the same. If you have a NaN bucket it must be + * the maximum value of the last position and all NaN entries will be counted + * in that bucket. */ def histogram(buckets: Array[Double]): Array[Long] = { - srdd.histogram(buckets.map(_.toDouble)) + srdd.histogram(buckets.map(_.toDouble), false) + } + + def histogram(buckets: Array[Double], evenBuckets: Boolean): Array[Long] = { + srdd.histogram(buckets.map(_.toDouble), evenBuckets) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 776a83cefe..33738ee094 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -83,44 +83,50 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * Compute a histogram of the data using bucketCount number of buckets evenly * spaced between the minimum and maximum of the RDD. For example if the min * value is 0 and the max is 100 and there are two buckets the resulting - * buckets will be [0,50) [50,100]. bucketCount must be at least 1 + * buckets will be [0, 50) [50, 100]. bucketCount must be at least 1 * If the RDD contains infinity, NaN throws an exception - * If the elements in RDD do not vary (max == min) throws an exception + * If the elements in RDD do not vary (max == min) always returns a single bucket. */ def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = { // Compute the minimum and the maxium val (max: Double, min: Double) = self.mapPartitions { items => Iterator(items.foldRight(-1/0.0, Double.NaN)((e: Double, x: Pair[Double, Double]) => - (x._1.max(e),x._2.min(e)))) + (x._1.max(e), x._2.min(e)))) }.reduce { (maxmin1, maxmin2) => (maxmin1._1.max(maxmin2._1), maxmin1._2.min(maxmin2._2)) } if (max.isNaN() || max.isInfinity || min.isInfinity ) { - throw new UnsupportedOperationException("Histogram on either an empty RDD or RDD containing +-infinity or NaN") - } - if (max == min) { - throw new UnsupportedOperationException("Histogram with no range in elements") + throw new UnsupportedOperationException( + "Histogram on either an empty RDD or RDD containing +/-infinity or NaN") } val increment: Double = (max-min)/bucketCount.toDouble - val range = Range.Double.inclusive(min, max, increment) + val range = if (increment != 0) { + Range.Double.inclusive(min, max, increment) + } else { + List(min, min) + } val buckets: Array[Double] = range.toArray - (buckets,histogram(buckets)) + (buckets, histogram(buckets, true)) } + /** * Compute a histogram using the provided buckets. The buckets are all open * to the left except for the last which is closed * e.g. for the array - * [1,10,20,50] the buckets are [1,10) [10,20) [20,50] + * [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50] * e.g 1<=x<10 , 10<=x<20, 20<=x<50 - * And on the input of 1 and 50 we would have a histogram of 1,0,0 + * And on the input of 1 and 50 we would have a histogram of 1, 0, 0 * - * Note: if your histogram is evenly spaced (e.g. [0,10,20,30]) this switches - * from an O(log n) inseration to O(1) per element. (where n = # buckets) + * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched + * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets + * to true. * buckets must be sorted and not contain any duplicates. * buckets array must be at least two elements - * All NaN entries are treated the same. + * All NaN entries are treated the same. If you have a NaN bucket it must be + * the maximum value of the last position and all NaN entries will be counted + * in that bucket. */ - def histogram(buckets: Array[Double]): Array[Long] = { + def histogram(buckets: Array[Double], evenBuckets: Boolean = false): Array[Long] = { if (buckets.length < 2) { throw new IllegalArgumentException("buckets array must have at least two elements") } @@ -129,11 +135,12 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { // to increment or returns None if there is no bucket. This is done so we can // specialize for uniformly distributed buckets and save the O(log n) binary // search cost. - def histogramPartition(bucketFunction: (Double) => Option[Int])(iter: Iterator[Double]): Iterator[Array[Long]] = { - val counters = new Array[Long](buckets.length-1) + def histogramPartition(bucketFunction: (Double) => Option[Int])(iter: Iterator[Double]): + Iterator[Array[Long]] = { + val counters = new Array[Long](buckets.length - 1) while (iter.hasNext) { bucketFunction(iter.next()) match { - case Some(x: Int) => {counters(x)+=1} + case Some(x: Int) => {counters(x) += 1} case _ => {} } } @@ -161,12 +168,12 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { } else { None } - } else if (location < buckets.length-1) { + } else if (location < buckets.length - 1) { // Exact match, just insert here Some(location) } else { // Exact match to the last element - Some(location-1) + Some(location - 1) } } // Determine the bucket function in constant time. Requires that buckets are evenly spaced @@ -175,34 +182,19 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { if (e.isNaN()) { return None } - val bucketNumber = (e-min)/(increment) + val bucketNumber = (e - min)/(increment) // We do this rather than buckets.lengthCompare(bucketNumber) // because Array[Double] fails to override it (for now). if (bucketNumber > count || bucketNumber < 0) { None } else { - Some(bucketNumber.toInt.min(count-1)) - } - } - def evenlySpaced(buckets: Array[Double]): Boolean = { - val delta = buckets(1)-buckets(0) - // Technically you could have an evenly spaced bucket with NaN - // increments but then its a single bucket and this makes the - // fastBucketFunction simpler. - if (delta.isNaN() || delta.isInfinite()) { - return false - } - for (i <- 1 to buckets.length-1) { - if (buckets(i)-buckets(i-1) != delta) { - return false - } + Some(bucketNumber.toInt.min(count - 1)) } - true } // Decide which bucket function to pass to histogramPartition. We decide here // rather than having a general function so that the decission need only be made // once rather than once per shard - val bucketFunction = if (evenlySpaced(buckets)) { + val bucketFunction = if (evenBuckets) { fastBucketFunction(buckets(0), buckets(1)-buckets(0), buckets.length-1) _ } else { basicBucketFunction _ diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index 2ec7173511..071084485a 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -34,134 +34,151 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { val rdd: RDD[Double] = sc.parallelize(Seq()) val buckets: Array[Double] = Array(0.0, 10.0) val histogramResults: Array[Long] = rdd.histogram(buckets) + val histogramResults2: Array[Long] = rdd.histogram(buckets, true) val expectedHistogramResults: Array[Long] = Array(0) assert(histogramResults === expectedHistogramResults) + assert(histogramResults2 === expectedHistogramResults) } test("WorksWithOutOfRangeWithOneBucket") { // Verify that if all of the elements are out of range the counts are zero - val rdd: RDD[Double] = sc.parallelize(Seq(10.01,-0.01)) + val rdd: RDD[Double] = sc.parallelize(Seq(10.01, -0.01)) val buckets: Array[Double] = Array(0.0, 10.0) val histogramResults: Array[Long] = rdd.histogram(buckets) + val histogramResults2: Array[Long] = rdd.histogram(buckets, true) val expectedHistogramResults: Array[Long] = Array(0) assert(histogramResults === expectedHistogramResults) + assert(histogramResults2 === expectedHistogramResults) } test("WorksInRangeWithOneBucket") { // Verify the basic case of one bucket and all elements in that bucket works - val rdd: RDD[Double] = sc.parallelize(Seq(1,2,3,4)) + val rdd: RDD[Double] = sc.parallelize(Seq(1, 2, 3, 4)) val buckets: Array[Double] = Array(0.0, 10.0) val histogramResults: Array[Long] = rdd.histogram(buckets) + val histogramResults2: Array[Long] = rdd.histogram(buckets, true) val expectedHistogramResults: Array[Long] = Array(4) assert(histogramResults === expectedHistogramResults) + assert(histogramResults2 === expectedHistogramResults) } test("WorksInRangeWithOneBucketExactMatch") { // Verify the basic case of one bucket and all elements in that bucket works - val rdd: RDD[Double] = sc.parallelize(Seq(1,2,3,4)) + val rdd: RDD[Double] = sc.parallelize(Seq(1, 2, 3, 4)) val buckets: Array[Double] = Array(1.0, 4.0) val histogramResults: Array[Long] = rdd.histogram(buckets) + val histogramResults2: Array[Long] = rdd.histogram(buckets, true) val expectedHistogramResults: Array[Long] = Array(4) assert(histogramResults === expectedHistogramResults) + assert(histogramResults2 === expectedHistogramResults) } test("WorksWithOutOfRangeWithTwoBuckets") { // Verify that out of range works with two buckets - val rdd: RDD[Double] = sc.parallelize(Seq(10.01,-0.01)) + val rdd: RDD[Double] = sc.parallelize(Seq(10.01, -0.01)) val buckets: Array[Double] = Array(0.0, 5.0, 10.0) val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(0,0) + val histogramResults2: Array[Long] = rdd.histogram(buckets, true) + val expectedHistogramResults: Array[Long] = Array(0, 0) assert(histogramResults === expectedHistogramResults) + assert(histogramResults2 === expectedHistogramResults) } test("WorksWithOutOfRangeWithTwoUnEvenBuckets") { // Verify that out of range works with two un even buckets - val rdd: RDD[Double] = sc.parallelize(Seq(10.01,-0.01)) + val rdd: RDD[Double] = sc.parallelize(Seq(10.01, -0.01)) val buckets: Array[Double] = Array(0.0, 4.0, 10.0) val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(0,0) + val expectedHistogramResults: Array[Long] = Array(0, 0) assert(histogramResults === expectedHistogramResults) } test("WorksInRangeWithTwoBuckets") { // Make sure that it works with two equally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(1,2,3,5,6)) + val rdd: RDD[Double] = sc.parallelize(Seq(1, 2, 3, 5, 6)) val buckets: Array[Double] = Array(0.0, 5.0, 10.0) val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(3,2) + val histogramResults2: Array[Long] = rdd.histogram(buckets, true) + val expectedHistogramResults: Array[Long] = Array(3, 2) assert(histogramResults === expectedHistogramResults) + assert(histogramResults2 === expectedHistogramResults) } test("WorksInRangeWithTwoBucketsAndNaN") { // Make sure that it works with two equally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(1,2,3,5,6,Double.NaN)) + val rdd: RDD[Double] = sc.parallelize(Seq(1, 2, 3, 5, 6, Double.NaN)) val buckets: Array[Double] = Array(0.0, 5.0, 10.0) val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(3,2) + val histogramResults2: Array[Long] = rdd.histogram(buckets, true) + val expectedHistogramResults: Array[Long] = Array(3, 2) assert(histogramResults === expectedHistogramResults) + assert(histogramResults2 === expectedHistogramResults) } test("WorksInRangeWithTwoUnevenBuckets") { // Make sure that it works with two unequally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(1,2,3,5,6)) + val rdd: RDD[Double] = sc.parallelize(Seq(1, 2, 3, 5, 6)) val buckets: Array[Double] = Array(0.0, 5.0, 11.0) val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(3,2) + val expectedHistogramResults: Array[Long] = Array(3, 2) assert(histogramResults === expectedHistogramResults) } test("WorksMixedRangeWithTwoUnevenBuckets") { // Make sure that it works with two unequally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(-0.01,0.0,1,2,3,5,6,11.0,11.01)) + val rdd: RDD[Double] = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01)) val buckets: Array[Double] = Array(0.0, 5.0, 11.0) val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(4,3) + val expectedHistogramResults: Array[Long] = Array(4, 3) assert(histogramResults === expectedHistogramResults) } test("WorksMixedRangeWithFourUnevenBuckets") { // Make sure that it works with two unequally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(-0.01,0.0,1,2,3,5,6,11.01,12.0,199.0,200.0,200.1)) - val buckets: Array[Double] = Array(0.0, 5.0, 11.0,12.0,200.0) + val rdd: RDD[Double] = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, + 200.0, 200.1)) + val buckets: Array[Double] = Array(0.0, 5.0, 11.0, 12.0, 200.0) val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(4,2,1,3) + val expectedHistogramResults: Array[Long] = Array(4, 2, 1, 3) assert(histogramResults === expectedHistogramResults) } test("WorksMixedRangeWithUnevenBucketsAndNaN") { // Make sure that it works with two unequally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(-0.01,0.0,1,2,3,5,6,11.01,12.0,199.0,200.0,200.1,Double.NaN)) - val buckets: Array[Double] = Array(0.0, 5.0, 11.0,12.0,200.0) + val rdd: RDD[Double] = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, + 200.0, 200.1, Double.NaN)) + val buckets: Array[Double] = Array(0.0, 5.0, 11.0, 12.0, 200.0) val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(4,2,1,3) + val expectedHistogramResults: Array[Long] = Array(4, 2, 1, 3) assert(histogramResults === expectedHistogramResults) } // Make sure this works with a NaN end bucket test("WorksMixedRangeWithUnevenBucketsAndNaNAndNaNRange") { // Make sure that it works with two unequally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(-0.01,0.0,1,2,3,5,6,11.01,12.0,199.0,200.0,200.1,Double.NaN)) - val buckets: Array[Double] = Array(0.0, 5.0, 11.0,12.0,200.0,Double.NaN) + val rdd: RDD[Double] = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, + 200.0, 200.1, Double.NaN)) + val buckets: Array[Double] = Array(0.0, 5.0, 11.0, 12.0, 200.0, Double.NaN) val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(4,2,1,2,3) + val expectedHistogramResults: Array[Long] = Array(4, 2, 1, 2, 3) assert(histogramResults === expectedHistogramResults) } // Make sure this works with a NaN end bucket and an inifity test("WorksMixedRangeWithUnevenBucketsAndNaNAndNaNRangeAndInfity") { // Make sure that it works with two unequally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(-0.01,0.0,1,2,3,5,6,11.01,12.0,199.0,200.0,200.1,1.0/0.0,-1.0/0.0,Double.NaN)) - val buckets: Array[Double] = Array(0.0, 5.0, 11.0,12.0,200.0,Double.NaN) + val rdd: RDD[Double] = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, + 200.0, 200.1, 1.0/0.0, -1.0/0.0, Double.NaN)) + val buckets: Array[Double] = Array(0.0, 5.0, 11.0, 12.0, 200.0, Double.NaN) val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(4,2,1,2,4) + val expectedHistogramResults: Array[Long] = Array(4, 2, 1, 2, 4) assert(histogramResults === expectedHistogramResults) } test("WorksWithOutOfRangeWithInfiniteBuckets") { // Verify that out of range works with two buckets - val rdd: RDD[Double] = sc.parallelize(Seq(10.01,-0.01,Double.NaN)) - val buckets: Array[Double] = Array(-1.0/0.0 ,0.0, 1.0/0.0) + val rdd: RDD[Double] = sc.parallelize(Seq(10.01, -0.01, Double.NaN)) + val buckets: Array[Double] = Array(-1.0/0.0 , 0.0, 1.0/0.0) val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(1,1) + val expectedHistogramResults: Array[Long] = Array(1, 1) assert(histogramResults === expectedHistogramResults) } // Test the failure mode with an invalid bucket array test("ThrowsExceptionOnInvalidBucketArray") { val rdd: RDD[Double] = sc.parallelize(Seq(1.0)) // Empty array - intercept[IllegalArgumentException]{ + intercept[IllegalArgumentException] { val buckets: Array[Double] = Array.empty[Double] val result = rdd.histogram(buckets) } // Single element array - intercept[IllegalArgumentException] - { + intercept[IllegalArgumentException] { val buckets: Array[Double] = Array(1.0) val result = rdd.histogram(buckets) } @@ -170,25 +187,45 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { // Test automatic histogram function test("WorksWithoutBucketsBasic") { // Verify the basic case of one bucket and all elements in that bucket works - val rdd: RDD[Double] = sc.parallelize(Seq(1,2,3,4)) + val rdd: RDD[Double] = sc.parallelize(Seq(1, 2, 3, 4)) val (histogramBuckets, histogramResults) = rdd.histogram(1) val expectedHistogramResults: Array[Long] = Array(4) - val expectedHistogramBuckets: Array[Double] = Array(1.0,4.0) + val expectedHistogramBuckets: Array[Double] = Array(1.0, 4.0) + assert(histogramResults === expectedHistogramResults) + assert(histogramBuckets === expectedHistogramBuckets) + } + // Test automatic histogram function with a single element + test("WorksWithoutBucketsBasicSingleElement") { + // Verify the basic case of one bucket and all elements in that bucket works + val rdd: RDD[Double] = sc.parallelize(Seq(1)) + val (histogramBuckets, histogramResults) = rdd.histogram(1) + val expectedHistogramResults: Array[Long] = Array(1) + val expectedHistogramBuckets: Array[Double] = Array(1.0, 1.0) + assert(histogramResults === expectedHistogramResults) + assert(histogramBuckets === expectedHistogramBuckets) + } + // Test automatic histogram function with a single element + test("WorksWithoutBucketsBasicNoRange") { + // Verify the basic case of one bucket and all elements in that bucket works + val rdd: RDD[Double] = sc.parallelize(Seq(1, 1, 1, 1)) + val (histogramBuckets, histogramResults) = rdd.histogram(1) + val expectedHistogramResults: Array[Long] = Array(4) + val expectedHistogramBuckets: Array[Double] = Array(1.0, 1.0) assert(histogramResults === expectedHistogramResults) assert(histogramBuckets === expectedHistogramBuckets) } test("WorksWithoutBucketsBasicTwo") { // Verify the basic case of one bucket and all elements in that bucket works - val rdd: RDD[Double] = sc.parallelize(Seq(1,2,3,4)) + val rdd: RDD[Double] = sc.parallelize(Seq(1, 2, 3, 4)) val (histogramBuckets, histogramResults) = rdd.histogram(2) - val expectedHistogramResults: Array[Long] = Array(2,2) - val expectedHistogramBuckets: Array[Double] = Array(1.0,2.5,4.0) + val expectedHistogramResults: Array[Long] = Array(2, 2) + val expectedHistogramBuckets: Array[Double] = Array(1.0, 2.5, 4.0) assert(histogramResults === expectedHistogramResults) assert(histogramBuckets === expectedHistogramBuckets) } test("WorksWithoutBucketsWithMoreRequestedThanElements") { // Verify the basic case of one bucket and all elements in that bucket works - val rdd: RDD[Double] = sc.parallelize(Seq(1,2)) + val rdd: RDD[Double] = sc.parallelize(Seq(1, 2)) val (histogramBuckets, histogramResults) = rdd.histogram(10) val expectedHistogramResults: Array[Long] = Array(1, 0, 0, 0, 0, 0, 0, 0, 0, 1) @@ -197,37 +234,24 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramResults === expectedHistogramResults) assert(histogramBuckets === expectedHistogramBuckets) } + // Test the failure mode with an invalid RDD test("ThrowsExceptionOnInvalidRDDs") { // infinity - intercept[UnsupportedOperationException]{ - val rdd: RDD[Double] = sc.parallelize(Seq(1,1.0/0.0)) + intercept[UnsupportedOperationException] { + val rdd: RDD[Double] = sc.parallelize(Seq(1, 1.0/0.0)) val result = rdd.histogram(1) } // NaN - intercept[UnsupportedOperationException] - { - val rdd: RDD[Double] = sc.parallelize(Seq(1,Double.NaN)) + intercept[UnsupportedOperationException] { + val rdd: RDD[Double] = sc.parallelize(Seq(1, Double.NaN)) val result = rdd.histogram(1) } // Empty - intercept[UnsupportedOperationException] - { + intercept[UnsupportedOperationException] { val rdd: RDD[Double] = sc.parallelize(Seq()) val result = rdd.histogram(1) } - // Single element - intercept[UnsupportedOperationException] - { - val rdd: RDD[Double] = sc.parallelize(Seq(1)) - val result = rdd.histogram(1) - } - // No Range - intercept[UnsupportedOperationException] - { - val rdd: RDD[Double] = sc.parallelize(Seq(1,1,1)) - val result = rdd.histogram(1) - } } } -- cgit v1.2.3 From 092b87e7c8f723a0c4ecf1dfb5379cad4c39d37f Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 21 Oct 2013 00:20:15 -0700 Subject: Remove extranious type definitions from inside of tests --- .../org/apache/spark/rdd/DoubleRDDSuite.scala | 172 ++++++++++----------- 1 file changed, 86 insertions(+), 86 deletions(-) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index 071084485a..0d8ac19024 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -32,154 +32,154 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { test("WorksOnEmpty") { // Make sure that it works on an empty input val rdd: RDD[Double] = sc.parallelize(Seq()) - val buckets: Array[Double] = Array(0.0, 10.0) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val histogramResults2: Array[Long] = rdd.histogram(buckets, true) - val expectedHistogramResults: Array[Long] = Array(0) + val buckets = Array(0.0, 10.0) + val histogramResults = rdd.histogram(buckets) + val histogramResults2 = rdd.histogram(buckets, true) + val expectedHistogramResults = Array(0) assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) } test("WorksWithOutOfRangeWithOneBucket") { // Verify that if all of the elements are out of range the counts are zero - val rdd: RDD[Double] = sc.parallelize(Seq(10.01, -0.01)) - val buckets: Array[Double] = Array(0.0, 10.0) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val histogramResults2: Array[Long] = rdd.histogram(buckets, true) - val expectedHistogramResults: Array[Long] = Array(0) + val rdd = sc.parallelize(Seq(10.01, -0.01)) + val buckets = Array(0.0, 10.0) + val histogramResults = rdd.histogram(buckets) + val histogramResults2 = rdd.histogram(buckets, true) + val expectedHistogramResults = Array(0) assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) } test("WorksInRangeWithOneBucket") { // Verify the basic case of one bucket and all elements in that bucket works - val rdd: RDD[Double] = sc.parallelize(Seq(1, 2, 3, 4)) - val buckets: Array[Double] = Array(0.0, 10.0) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val histogramResults2: Array[Long] = rdd.histogram(buckets, true) - val expectedHistogramResults: Array[Long] = Array(4) + val rdd = sc.parallelize(Seq(1, 2, 3, 4)) + val buckets = Array(0.0, 10.0) + val histogramResults = rdd.histogram(buckets) + val histogramResults2 = rdd.histogram(buckets, true) + val expectedHistogramResults = Array(4) assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) } test("WorksInRangeWithOneBucketExactMatch") { // Verify the basic case of one bucket and all elements in that bucket works - val rdd: RDD[Double] = sc.parallelize(Seq(1, 2, 3, 4)) - val buckets: Array[Double] = Array(1.0, 4.0) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val histogramResults2: Array[Long] = rdd.histogram(buckets, true) - val expectedHistogramResults: Array[Long] = Array(4) + val rdd = sc.parallelize(Seq(1, 2, 3, 4)) + val buckets = Array(1.0, 4.0) + val histogramResults = rdd.histogram(buckets) + val histogramResults2 = rdd.histogram(buckets, true) + val expectedHistogramResults = Array(4) assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) } test("WorksWithOutOfRangeWithTwoBuckets") { // Verify that out of range works with two buckets - val rdd: RDD[Double] = sc.parallelize(Seq(10.01, -0.01)) - val buckets: Array[Double] = Array(0.0, 5.0, 10.0) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val histogramResults2: Array[Long] = rdd.histogram(buckets, true) - val expectedHistogramResults: Array[Long] = Array(0, 0) + val rdd = sc.parallelize(Seq(10.01, -0.01)) + val buckets = Array(0.0, 5.0, 10.0) + val histogramResults = rdd.histogram(buckets) + val histogramResults2 = rdd.histogram(buckets, true) + val expectedHistogramResults = Array(0, 0) assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) } test("WorksWithOutOfRangeWithTwoUnEvenBuckets") { // Verify that out of range works with two un even buckets - val rdd: RDD[Double] = sc.parallelize(Seq(10.01, -0.01)) - val buckets: Array[Double] = Array(0.0, 4.0, 10.0) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(0, 0) + val rdd = sc.parallelize(Seq(10.01, -0.01)) + val buckets = Array(0.0, 4.0, 10.0) + val histogramResults = rdd.histogram(buckets) + val expectedHistogramResults = Array(0, 0) assert(histogramResults === expectedHistogramResults) } test("WorksInRangeWithTwoBuckets") { // Make sure that it works with two equally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(1, 2, 3, 5, 6)) - val buckets: Array[Double] = Array(0.0, 5.0, 10.0) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val histogramResults2: Array[Long] = rdd.histogram(buckets, true) - val expectedHistogramResults: Array[Long] = Array(3, 2) + val rdd = sc.parallelize(Seq(1, 2, 3, 5, 6)) + val buckets = Array(0.0, 5.0, 10.0) + val histogramResults = rdd.histogram(buckets) + val histogramResults2 = rdd.histogram(buckets, true) + val expectedHistogramResults = Array(3, 2) assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) } test("WorksInRangeWithTwoBucketsAndNaN") { // Make sure that it works with two equally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(1, 2, 3, 5, 6, Double.NaN)) - val buckets: Array[Double] = Array(0.0, 5.0, 10.0) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val histogramResults2: Array[Long] = rdd.histogram(buckets, true) - val expectedHistogramResults: Array[Long] = Array(3, 2) + val rdd = sc.parallelize(Seq(1, 2, 3, 5, 6, Double.NaN)) + val buckets = Array(0.0, 5.0, 10.0) + val histogramResults = rdd.histogram(buckets) + val histogramResults2 = rdd.histogram(buckets, true) + val expectedHistogramResults = Array(3, 2) assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) } test("WorksInRangeWithTwoUnevenBuckets") { // Make sure that it works with two unequally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(1, 2, 3, 5, 6)) - val buckets: Array[Double] = Array(0.0, 5.0, 11.0) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(3, 2) + val rdd = sc.parallelize(Seq(1, 2, 3, 5, 6)) + val buckets = Array(0.0, 5.0, 11.0) + val histogramResults = rdd.histogram(buckets) + val expectedHistogramResults = Array(3, 2) assert(histogramResults === expectedHistogramResults) } test("WorksMixedRangeWithTwoUnevenBuckets") { // Make sure that it works with two unequally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01)) - val buckets: Array[Double] = Array(0.0, 5.0, 11.0) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(4, 3) + val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01)) + val buckets = Array(0.0, 5.0, 11.0) + val histogramResults = rdd.histogram(buckets) + val expectedHistogramResults = Array(4, 3) assert(histogramResults === expectedHistogramResults) } test("WorksMixedRangeWithFourUnevenBuckets") { // Make sure that it works with two unequally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, + val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1)) - val buckets: Array[Double] = Array(0.0, 5.0, 11.0, 12.0, 200.0) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(4, 2, 1, 3) + val buckets = Array(0.0, 5.0, 11.0, 12.0, 200.0) + val histogramResults = rdd.histogram(buckets) + val expectedHistogramResults = Array(4, 2, 1, 3) assert(histogramResults === expectedHistogramResults) } test("WorksMixedRangeWithUnevenBucketsAndNaN") { // Make sure that it works with two unequally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, + val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1, Double.NaN)) - val buckets: Array[Double] = Array(0.0, 5.0, 11.0, 12.0, 200.0) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(4, 2, 1, 3) + val buckets = Array(0.0, 5.0, 11.0, 12.0, 200.0) + val histogramResults = rdd.histogram(buckets) + val expectedHistogramResults = Array(4, 2, 1, 3) assert(histogramResults === expectedHistogramResults) } // Make sure this works with a NaN end bucket test("WorksMixedRangeWithUnevenBucketsAndNaNAndNaNRange") { // Make sure that it works with two unequally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, + val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1, Double.NaN)) - val buckets: Array[Double] = Array(0.0, 5.0, 11.0, 12.0, 200.0, Double.NaN) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(4, 2, 1, 2, 3) + val buckets = Array(0.0, 5.0, 11.0, 12.0, 200.0, Double.NaN) + val histogramResults = rdd.histogram(buckets) + val expectedHistogramResults = Array(4, 2, 1, 2, 3) assert(histogramResults === expectedHistogramResults) } // Make sure this works with a NaN end bucket and an inifity test("WorksMixedRangeWithUnevenBucketsAndNaNAndNaNRangeAndInfity") { // Make sure that it works with two unequally spaced buckets and elements in each - val rdd: RDD[Double] = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, + val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1, 1.0/0.0, -1.0/0.0, Double.NaN)) - val buckets: Array[Double] = Array(0.0, 5.0, 11.0, 12.0, 200.0, Double.NaN) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(4, 2, 1, 2, 4) + val buckets = Array(0.0, 5.0, 11.0, 12.0, 200.0, Double.NaN) + val histogramResults = rdd.histogram(buckets) + val expectedHistogramResults = Array(4, 2, 1, 2, 4) assert(histogramResults === expectedHistogramResults) } test("WorksWithOutOfRangeWithInfiniteBuckets") { // Verify that out of range works with two buckets - val rdd: RDD[Double] = sc.parallelize(Seq(10.01, -0.01, Double.NaN)) - val buckets: Array[Double] = Array(-1.0/0.0 , 0.0, 1.0/0.0) - val histogramResults: Array[Long] = rdd.histogram(buckets) - val expectedHistogramResults: Array[Long] = Array(1, 1) + val rdd = sc.parallelize(Seq(10.01, -0.01, Double.NaN)) + val buckets = Array(-1.0/0.0 , 0.0, 1.0/0.0) + val histogramResults = rdd.histogram(buckets) + val expectedHistogramResults = Array(1, 1) assert(histogramResults === expectedHistogramResults) } // Test the failure mode with an invalid bucket array test("ThrowsExceptionOnInvalidBucketArray") { - val rdd: RDD[Double] = sc.parallelize(Seq(1.0)) + val rdd = sc.parallelize(Seq(1.0)) // Empty array intercept[IllegalArgumentException] { - val buckets: Array[Double] = Array.empty[Double] + val buckets = Array.empty[Double] val result = rdd.histogram(buckets) } // Single element array intercept[IllegalArgumentException] { - val buckets: Array[Double] = Array(1.0) + val buckets = Array(1.0) val result = rdd.histogram(buckets) } } @@ -187,49 +187,49 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { // Test automatic histogram function test("WorksWithoutBucketsBasic") { // Verify the basic case of one bucket and all elements in that bucket works - val rdd: RDD[Double] = sc.parallelize(Seq(1, 2, 3, 4)) + val rdd = sc.parallelize(Seq(1, 2, 3, 4)) val (histogramBuckets, histogramResults) = rdd.histogram(1) - val expectedHistogramResults: Array[Long] = Array(4) - val expectedHistogramBuckets: Array[Double] = Array(1.0, 4.0) + val expectedHistogramResults = Array(4) + val expectedHistogramBuckets = Array(1.0, 4.0) assert(histogramResults === expectedHistogramResults) assert(histogramBuckets === expectedHistogramBuckets) } // Test automatic histogram function with a single element test("WorksWithoutBucketsBasicSingleElement") { // Verify the basic case of one bucket and all elements in that bucket works - val rdd: RDD[Double] = sc.parallelize(Seq(1)) + val rdd = sc.parallelize(Seq(1)) val (histogramBuckets, histogramResults) = rdd.histogram(1) - val expectedHistogramResults: Array[Long] = Array(1) - val expectedHistogramBuckets: Array[Double] = Array(1.0, 1.0) + val expectedHistogramResults = Array(1) + val expectedHistogramBuckets = Array(1.0, 1.0) assert(histogramResults === expectedHistogramResults) assert(histogramBuckets === expectedHistogramBuckets) } // Test automatic histogram function with a single element test("WorksWithoutBucketsBasicNoRange") { // Verify the basic case of one bucket and all elements in that bucket works - val rdd: RDD[Double] = sc.parallelize(Seq(1, 1, 1, 1)) + val rdd = sc.parallelize(Seq(1, 1, 1, 1)) val (histogramBuckets, histogramResults) = rdd.histogram(1) - val expectedHistogramResults: Array[Long] = Array(4) - val expectedHistogramBuckets: Array[Double] = Array(1.0, 1.0) + val expectedHistogramResults = Array(4) + val expectedHistogramBuckets = Array(1.0, 1.0) assert(histogramResults === expectedHistogramResults) assert(histogramBuckets === expectedHistogramBuckets) } test("WorksWithoutBucketsBasicTwo") { // Verify the basic case of one bucket and all elements in that bucket works - val rdd: RDD[Double] = sc.parallelize(Seq(1, 2, 3, 4)) + val rdd = sc.parallelize(Seq(1, 2, 3, 4)) val (histogramBuckets, histogramResults) = rdd.histogram(2) - val expectedHistogramResults: Array[Long] = Array(2, 2) - val expectedHistogramBuckets: Array[Double] = Array(1.0, 2.5, 4.0) + val expectedHistogramResults = Array(2, 2) + val expectedHistogramBuckets = Array(1.0, 2.5, 4.0) assert(histogramResults === expectedHistogramResults) assert(histogramBuckets === expectedHistogramBuckets) } test("WorksWithoutBucketsWithMoreRequestedThanElements") { // Verify the basic case of one bucket and all elements in that bucket works - val rdd: RDD[Double] = sc.parallelize(Seq(1, 2)) + val rdd = sc.parallelize(Seq(1, 2)) val (histogramBuckets, histogramResults) = rdd.histogram(10) - val expectedHistogramResults: Array[Long] = + val expectedHistogramResults = Array(1, 0, 0, 0, 0, 0, 0, 0, 0, 1) - val expectedHistogramBuckets: Array[Double] = + val expectedHistogramBuckets = Array(1.0, 1.1, 1.2, 1.3, 1.4, 1.5, 1.6, 1.7, 1.8, 1.9, 2.0) assert(histogramResults === expectedHistogramResults) assert(histogramBuckets === expectedHistogramBuckets) @@ -239,12 +239,12 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { test("ThrowsExceptionOnInvalidRDDs") { // infinity intercept[UnsupportedOperationException] { - val rdd: RDD[Double] = sc.parallelize(Seq(1, 1.0/0.0)) + val rdd = sc.parallelize(Seq(1, 1.0/0.0)) val result = rdd.histogram(1) } // NaN intercept[UnsupportedOperationException] { - val rdd: RDD[Double] = sc.parallelize(Seq(1, Double.NaN)) + val rdd = sc.parallelize(Seq(1, Double.NaN)) val result = rdd.histogram(1) } // Empty -- cgit v1.2.3 From 20b33bc4b5de1addd943c7a1e6d5d2366d9cd445 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 21 Oct 2013 00:21:37 -0700 Subject: Remove extranious type declerations --- core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 33738ee094..02d75eccc5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -99,13 +99,13 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { throw new UnsupportedOperationException( "Histogram on either an empty RDD or RDD containing +/-infinity or NaN") } - val increment: Double = (max-min)/bucketCount.toDouble + val increment = (max-min)/bucketCount.toDouble val range = if (increment != 0) { Range.Double.inclusive(min, max, increment) } else { List(min, min) } - val buckets: Array[Double] = range.toArray + val buckets = range.toArray (buckets, histogram(buckets, true)) } -- cgit v1.2.3 From 5e91495f5c718c837b5a5af2268f6faad00d357f Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Wed, 30 Oct 2013 17:07:24 -0700 Subject: Deduplicate Local and Cluster schedulers. The code in LocalScheduler/LocalTaskSetManager was nearly identical to the code in ClusterScheduler/ClusterTaskSetManager. The redundancy made making updating the schedulers unnecessarily painful and error- prone. This commit combines the two into a single TaskScheduler/ TaskSetManager. --- .../main/scala/org/apache/spark/SparkContext.scala | 34 +- .../spark/scheduler/ExecutorLossReason.scala | 38 ++ .../apache/spark/scheduler/SchedulerBackend.scala | 37 ++ .../apache/spark/scheduler/TaskResultGetter.scala | 107 ++++ .../org/apache/spark/scheduler/TaskScheduler.scala | 480 +++++++++++++- .../apache/spark/scheduler/TaskSetManager.scala | 688 +++++++++++++++++++- .../org/apache/spark/scheduler/WorkerOffer.scala | 24 + .../spark/scheduler/cluster/ClusterScheduler.scala | 486 -------------- .../scheduler/cluster/ClusterTaskSetManager.scala | 703 --------------------- .../cluster/CoarseGrainedSchedulerBackend.scala | 5 +- .../scheduler/cluster/ExecutorLossReason.scala | 38 -- .../spark/scheduler/cluster/SchedulerBackend.scala | 37 -- .../scheduler/cluster/SimrSchedulerBackend.scala | 4 +- .../cluster/SparkDeploySchedulerBackend.scala | 6 +- .../spark/scheduler/cluster/TaskResultGetter.scala | 108 ---- .../spark/scheduler/cluster/WorkerOffer.scala | 24 - .../mesos/CoarseMesosSchedulerBackend.scala | 5 +- .../cluster/mesos/MesosSchedulerBackend.scala | 9 +- .../spark/scheduler/local/LocalBackend.scala | 73 +++ .../spark/scheduler/local/LocalScheduler.scala | 219 ------- .../scheduler/local/LocalTaskSetManager.scala | 191 ------ .../test/scala/org/apache/spark/FailureSuite.scala | 20 +- .../spark/scheduler/SparkListenerSuite.scala | 19 +- .../scheduler/cluster/TaskResultGetterSuite.scala | 4 +- .../scheduler/local/LocalSchedulerSuite.scala | 227 ------- .../scheduler/cluster/YarnClusterScheduler.scala | 10 +- 26 files changed, 1472 insertions(+), 2124 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorLossReason.scala delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/WorkerOffer.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala delete mode 100644 core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ade75e20d5..1850436ff2 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -56,10 +56,9 @@ import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, - SparkDeploySchedulerBackend, ClusterScheduler, SimrSchedulerBackend} + SparkDeploySchedulerBackend, SimrSchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import org.apache.spark.scheduler.local.LocalScheduler -import org.apache.spark.scheduler.StageInfo +import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, @@ -149,8 +148,6 @@ class SparkContext( private[spark] var taskScheduler: TaskScheduler = { // Regular expression used for local[N] master format val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r - // Regular expression for local[N, maxRetries], used in tests with failing tasks - val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r // Regular expression for simulating a Spark cluster of [N, cores, memory] locally val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r // Regular expression for connecting to Spark deploy clusters @@ -162,23 +159,26 @@ class SparkContext( master match { case "local" => - new LocalScheduler(1, 0, this) + val scheduler = new TaskScheduler(this) + val backend = new LocalBackend(scheduler, 1) + scheduler.initialize(backend) + scheduler case LOCAL_N_REGEX(threads) => - new LocalScheduler(threads.toInt, 0, this) - - case LOCAL_N_FAILURES_REGEX(threads, maxFailures) => - new LocalScheduler(threads.toInt, maxFailures.toInt, this) + val scheduler = new TaskScheduler(this) + val backend = new LocalBackend(scheduler, threads.toInt) + scheduler.initialize(backend) + scheduler case SPARK_REGEX(sparkUrl) => - val scheduler = new ClusterScheduler(this) + val scheduler = new TaskScheduler(this) val masterUrls = sparkUrl.split(",").map("spark://" + _) val backend = new SparkDeploySchedulerBackend(scheduler, this, masterUrls, appName) scheduler.initialize(backend) scheduler case SIMR_REGEX(simrUrl) => - val scheduler = new ClusterScheduler(this) + val scheduler = new TaskScheduler(this) val backend = new SimrSchedulerBackend(scheduler, this, simrUrl) scheduler.initialize(backend) scheduler @@ -192,7 +192,7 @@ class SparkContext( memoryPerSlaveInt, SparkContext.executorMemoryRequested)) } - val scheduler = new ClusterScheduler(this) + val scheduler = new TaskScheduler(this) val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val masterUrls = localCluster.start() @@ -207,7 +207,7 @@ class SparkContext( val scheduler = try { val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler") val cons = clazz.getConstructor(classOf[SparkContext]) - cons.newInstance(this).asInstanceOf[ClusterScheduler] + cons.newInstance(this).asInstanceOf[TaskScheduler] } catch { // TODO: Enumerate the exact reasons why it can fail // But irrespective of it, it means we cannot proceed ! @@ -221,7 +221,7 @@ class SparkContext( case MESOS_REGEX(mesosUrl) => MesosNativeLibrary.load() - val scheduler = new ClusterScheduler(this) + val scheduler = new TaskScheduler(this) val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean val backend = if (coarseGrained) { new CoarseMesosSchedulerBackend(scheduler, this, mesosUrl, appName) @@ -593,9 +593,7 @@ class SparkContext( } addedFiles(key) = System.currentTimeMillis - // Fetch the file locally in case a job is executed locally. - // Jobs that run through LocalScheduler will already fetch the required dependencies, - // but jobs run in DAGScheduler.runLocally() will not so we must fetch the files here. + // Fetch the file locally in case a job is executed using DAGScheduler.runLocally(). Utils.fetchFile(path, new File(SparkFiles.getRootDirectory)) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala new file mode 100644 index 0000000000..2bc43a9186 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 org.apache.spark.executor.ExecutorExitCode + +/** + * Represents an explanation for a executor or whole slave failing or exiting. + */ +private[spark] +class ExecutorLossReason(val message: String) { + override def toString: String = message +} + +private[spark] +case class ExecutorExited(val exitCode: Int) + extends ExecutorLossReason(ExecutorExitCode.explainExitCode(exitCode)) { +} + +private[spark] +case class SlaveLost(_message: String = "Slave lost") + extends ExecutorLossReason(_message) { +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala new file mode 100644 index 0000000000..1f0839a0e1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 org.apache.spark.SparkContext + +/** + * A backend interface for scheduling systems that allows plugging in different ones under + * TaskScheduler. We assume a Mesos-like model where the application gets resource offers as + * machines become available and can launch tasks on them. + */ +private[spark] trait SchedulerBackend { + def start(): Unit + def stop(): Unit + def reviveOffers(): Unit + def defaultParallelism(): Int + + def killTask(taskId: Long, executorId: String): Unit = throw new UnsupportedOperationException + + // Memory used by each executor (in megabytes) + protected val executorMemory: Int = SparkContext.executorMemoryRequested +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala new file mode 100644 index 0000000000..5408fa7353 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 java.nio.ByteBuffer +import java.util.concurrent.{LinkedBlockingDeque, ThreadFactory, ThreadPoolExecutor, TimeUnit} + +import org.apache.spark._ +import org.apache.spark.TaskState.TaskState +import org.apache.spark.serializer.SerializerInstance +import org.apache.spark.util.Utils + +/** + * Runs a thread pool that deserializes and remotely fetches (if necessary) task results. + */ +private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskScheduler) + extends Logging { + private val THREADS = System.getProperty("spark.resultGetter.threads", "4").toInt + private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( + THREADS, "Result resolver thread") + + protected val serializer = new ThreadLocal[SerializerInstance] { + override def initialValue(): SerializerInstance = { + return sparkEnv.closureSerializer.newInstance() + } + } + + def enqueueSuccessfulTask( + taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) { + getTaskResultExecutor.execute(new Runnable { + override def run() { + try { + val result = serializer.get().deserialize[TaskResult[_]](serializedData) match { + case directResult: DirectTaskResult[_] => directResult + case IndirectTaskResult(blockId) => + logDebug("Fetching indirect task result for TID %s".format(tid)) + scheduler.handleTaskGettingResult(taskSetManager, tid) + val serializedTaskResult = sparkEnv.blockManager.getRemoteBytes(blockId) + if (!serializedTaskResult.isDefined) { + /* We won't be able to get the task result if the machine that ran the task failed + * between when the task ended and when we tried to fetch the result, or if the + * block manager had to flush the result. */ + scheduler.handleFailedTask( + taskSetManager, tid, TaskState.FINISHED, Some(TaskResultLost)) + return + } + val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]]( + serializedTaskResult.get) + sparkEnv.blockManager.master.removeBlock(blockId) + deserializedResult + } + result.metrics.resultSize = serializedData.limit() + scheduler.handleSuccessfulTask(taskSetManager, tid, result) + } catch { + case cnf: ClassNotFoundException => + val loader = Thread.currentThread.getContextClassLoader + taskSetManager.abort("ClassNotFound with classloader: " + loader) + case ex => + taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex)) + } + } + }) + } + + def enqueueFailedTask(taskSetManager: TaskSetManager, tid: Long, taskState: TaskState, + serializedData: ByteBuffer) { + var reason: Option[TaskEndReason] = None + getTaskResultExecutor.execute(new Runnable { + override def run() { + try { + if (serializedData != null && serializedData.limit() > 0) { + reason = Some(serializer.get().deserialize[TaskEndReason]( + serializedData, getClass.getClassLoader)) + } + } catch { + case cnd: ClassNotFoundException => + // Log an error but keep going here -- the task failed, so not catastropic if we can't + // deserialize the reason. + val loader = Thread.currentThread.getContextClassLoader + logError( + "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) + case ex => {} + } + scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) + } + }) + } + + def stop() { + getTaskResultExecutor.shutdownNow() + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 10e0478108..3f694dd25d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -17,39 +17,477 @@ package org.apache.spark.scheduler +import java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicLong +import java.util.{TimerTask, Timer} + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet + +import org.apache.spark._ +import org.apache.spark.TaskState.TaskState +import org.apache.spark.scheduler._ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode /** - * Low-level task scheduler interface, implemented by both ClusterScheduler and LocalScheduler. - * Each TaskScheduler schedulers task for a single SparkContext. - * These schedulers get sets of tasks submitted to them from the DAGScheduler for each stage, - * and are responsible for sending the tasks to the cluster, running them, retrying if there - * are failures, and mitigating stragglers. They return events to the DAGScheduler. + * Schedules tasks for a single SparkContext. Receives a set of tasks from the DAGScheduler for + * each stage, and is responsible for sending tasks to executors, running them, retrying if there + * are failures, and mitigating stragglers. Returns events to the DAGScheduler. + * + * Clients should first call 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 can also work with a local setup by using a LocalBackend and setting isLocal to true. + * 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] trait TaskScheduler { +private[spark] class TaskScheduler(val sc: SparkContext, isLocal: Boolean = false) extends 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 + + // TaskSetManagers are not thread safe, so any access to one should be synchronized + // on this class. + val activeTaskSets = new HashMap[String, TaskSetManager] + + val taskIdToTaskSetId = new HashMap[Long, String] + val taskIdToExecutorId = new HashMap[Long, String] + val taskSetTaskIds = new HashMap[String, HashSet[Long]] + + @volatile private var hasReceivedTask = false + @volatile private var hasLaunchedTask = false + private val starvationTimer = new Timer(true) + + // Incrementing task IDs + val nextTaskId = new AtomicLong(0) + + // Which executor IDs we have executors on + val activeExecutorIds = new 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 executorsByHost = new HashMap[String, HashSet[String]] + + private val executorIdToHost = new HashMap[String, String] + + // Listener object to pass upcalls into + var dagScheduler: DAGScheduler = null + + var backend: SchedulerBackend = null + + val mapOutputTracker = SparkEnv.get.mapOutputTracker + + var schedulableBuilder: SchedulableBuilder = null + var rootPool: Pool = null + // default scheduler is FIFO + val schedulingMode: SchedulingMode = SchedulingMode.withName( + System.getProperty("spark.scheduler.mode", "FIFO")) + + // This is a var so that we can reset it for testing purposes. + private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) + + def setDAGScheduler(dagScheduler: DAGScheduler) { + this.dagScheduler = dagScheduler + } + + def initialize(context: SchedulerBackend) { + backend = context + // temporarily set rootPool name to empty + rootPool = new Pool("", schedulingMode, 0, 0) + schedulableBuilder = { + schedulingMode match { + case SchedulingMode.FIFO => + new FIFOSchedulableBuilder(rootPool) + case SchedulingMode.FAIR => + new FairSchedulableBuilder(rootPool) + } + } + schedulableBuilder.buildPools() + } + + def newTaskId(): Long = nextTaskId.getAndIncrement() + + def start() { + backend.start() + + if (!isLocal && System.getProperty("spark.speculation", "false").toBoolean) { + new Thread("TaskScheduler speculation check") { + setDaemon(true) + + override def run() { + logInfo("Starting speculative execution thread") + while (true) { + try { + Thread.sleep(SPECULATION_INTERVAL) + } catch { + case e: InterruptedException => {} + } + checkSpeculatableTasks() + } + } + }.start() + } + } + + def submitTasks(taskSet: TaskSet) { + val tasks = taskSet.tasks + logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks") + this.synchronized { + val manager = new TaskSetManager(this, taskSet) + activeTaskSets(taskSet.id) = manager + schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) + taskSetTaskIds(taskSet.id) = new HashSet[Long]() + + if (!isLocal && !hasReceivedTask) { + starvationTimer.scheduleAtFixedRate(new TimerTask() { + override def run() { + if (!hasLaunchedTask) { + logWarning("Initial job has not accepted any resources; " + + "check your cluster UI to ensure that workers are registered " + + "and have sufficient memory") + } else { + this.cancel() + } + } + }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) + } + hasReceivedTask = true + } + backend.reviveOffers() + } - def rootPool: Pool + def cancelTasks(stageId: Int): Unit = synchronized { + logInfo("Cancelling stage " + stageId) + activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) => + // There are two possible cases here: + // 1. The task set manager has been created and some tasks have been scheduled. + // In this case, send a kill signal to the executors to kill the task and then abort + // the stage. + // 2. The task set manager has been created but no tasks has been scheduled. In this case, + // simply abort the stage. + val taskIds = taskSetTaskIds(tsm.taskSet.id) + if (taskIds.size > 0) { + taskIds.foreach { tid => + val execId = taskIdToExecutorId(tid) + backend.killTask(tid, execId) + } + } + tsm.error("Stage %d was cancelled".format(stageId)) + } + } - def schedulingMode: SchedulingMode + def taskSetFinished(manager: TaskSetManager): Unit = synchronized { + // Check to see if the given task set has been removed. This is possible in the case of + // multiple unrecoverable task failures (e.g. if the entire task set is killed when it has + // more than one running tasks). + if (activeTaskSets.contains(manager.taskSet.id)) { + activeTaskSets -= manager.taskSet.id + manager.parent.removeSchedulable(manager) + logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) + taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) + taskIdToExecutorId --= taskSetTaskIds(manager.taskSet.id) + taskSetTaskIds.remove(manager.taskSet.id) + } + } - def start(): Unit + /** + * Called by cluster manager to offer resources on slaves. We respond by asking our active task + * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so + * that tasks are balanced across the cluster. + */ + def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = synchronized { + SparkEnv.set(sc.env) - // Invoked after system has successfully initialized (typically in spark context). - // Yarn uses this to bootstrap allocation of resources based on preferred locations, wait for slave registerations, etc. + // 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) + } + } + + // 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 sortedTaskSets = rootPool.getSortedTaskSetQueue() + for (taskSet <- sortedTaskSets) { + logDebug("parentName: %s, name: %s, runningTasks: %s".format( + 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 (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 <- taskSet.resourceOffer(execId, host, availableCpus(i), maxLocality)) { + tasks(i) += task + val tid = task.taskId + taskIdToTaskSetId(tid) = taskSet.taskSet.id + taskSetTaskIds(taskSet.taskSet.id) += tid + taskIdToExecutorId(tid) = execId + activeExecutorIds += execId + executorsByHost(host) += execId + availableCpus(i) -= 1 + launchedTask = true + } + } + } while (launchedTask) + } + + if (tasks.size > 0) { + hasLaunchedTask = true + } + return tasks + } + + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + var failedExecutor: Option[String] = None + var taskFailed = false + synchronized { + try { + if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) { + // We lost this entire executor, so remember that it's gone + val execId = taskIdToExecutorId(tid) + if (activeExecutorIds.contains(execId)) { + removeExecutor(execId) + failedExecutor = Some(execId) + } + } + taskIdToTaskSetId.get(tid) match { + case Some(taskSetId) => + if (TaskState.isFinished(state)) { + taskIdToTaskSetId.remove(tid) + if (taskSetTaskIds.contains(taskSetId)) { + taskSetTaskIds(taskSetId) -= tid + } + taskIdToExecutorId.remove(tid) + } + if (state == TaskState.FAILED) { + taskFailed = true + } + activeTaskSets.get(taskSetId).foreach { taskSet => + if (state == TaskState.FINISHED) { + taskSet.removeRunningTask(tid) + taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) + } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { + taskSet.removeRunningTask(tid) + taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData) + } + } + case None => + logInfo("Ignoring update from TID " + tid + " because its task set is gone") + } + } catch { + case e: Exception => logError("Exception in statusUpdate", e) + } + } + // Update the DAGScheduler without holding a lock on this, since that can deadlock + if (failedExecutor != None) { + dagScheduler.executorLost(failedExecutor.get) + backend.reviveOffers() + } + if (taskFailed) { + // Also revive offers if a task had failed for some reason other than host lost + backend.reviveOffers() + } + } + + def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long) { + taskSetManager.handleTaskGettingResult(tid) + } + + def handleSuccessfulTask( + taskSetManager: TaskSetManager, + tid: Long, + taskResult: DirectTaskResult[_]) = synchronized { + taskSetManager.handleSuccessfulTask(tid, taskResult) + } + + def handleFailedTask( + taskSetManager: TaskSetManager, + tid: Long, + taskState: TaskState, + reason: Option[TaskEndReason]) = synchronized { + taskSetManager.handleFailedTask(tid, taskState, reason) + if (taskState == TaskState.FINISHED) { + // The task finished successfully but the result was lost, so we should revive offers. + backend.reviveOffers() + } + } + + def error(message: String) { + synchronized { + if (activeTaskSets.size > 0) { + // Have each task set throw a SparkException with the error + for ((taskSetId, manager) <- activeTaskSets) { + try { + manager.error(message) + } catch { + case e: Exception => logError("Exception in error callback", e) + } + } + } else { + // No task sets are active but we still got an error. Just exit since this + // must mean the error is during registration. + // It might be good to do something smarter here in the future. + logError("Exiting due to error from task scheduler: " + message) + System.exit(1) + } + } + } + + def stop() { + if (backend != null) { + backend.stop() + } + if (taskResultGetter != null) { + taskResultGetter.stop() + } + + // sleeping for an arbitrary 5 seconds : to ensure that messages are sent out. + // TODO: Do something better ! + Thread.sleep(5000L) + } + + def defaultParallelism() = backend.defaultParallelism() + + // Check for speculatable tasks in all our active jobs. + def checkSpeculatableTasks() { + var shouldRevive = false + synchronized { + shouldRevive = rootPool.checkSpeculatableTasks() + } + if (shouldRevive) { + backend.reviveOffers() + } + } + + // Check for pending tasks in all our active jobs. + def hasPendingTasks: Boolean = { + synchronized { + rootPool.hasPendingTasks() + } + } + + def executorLost(executorId: String, reason: ExecutorLossReason) { + var failedExecutor: Option[String] = None + + synchronized { + if (activeExecutorIds.contains(executorId)) { + val hostPort = executorIdToHost(executorId) + logError("Lost executor %s on %s: %s".format(executorId, hostPort, reason)) + removeExecutor(executorId) + failedExecutor = Some(executorId) + } else { + // We may get multiple executorLost() calls with different loss reasons. For example, one + // may be triggered by a dropped connection from the slave while another may be a report + // of executor termination from Mesos. We produce log messages for both so we eventually + // report the termination reason. + logError("Lost an executor " + executorId + " (already removed): " + reason) + } + } + // Call dagScheduler.executorLost without holding the lock on this to prevent deadlock + if (failedExecutor != None) { + dagScheduler.executorLost(failedExecutor.get) + backend.reviveOffers() + } + } + + /** Remove an executor from all our data structures and mark it as lost */ + private def removeExecutor(executorId: String) { + activeExecutorIds -= executorId + val host = executorIdToHost(executorId) + val execs = executorsByHost.getOrElse(host, new HashSet) + execs -= executorId + if (execs.isEmpty) { + executorsByHost -= host + } + executorIdToHost -= executorId + rootPool.executorLost(executorId, host) + } + + def executorGained(execId: String, host: String) { + dagScheduler.executorGained(execId, host) + } + + def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized { + executorsByHost.get(host).map(_.toSet) + } + + def hasExecutorsAliveOnHost(host: String): Boolean = synchronized { + executorsByHost.contains(host) + } + + def isExecutorAlive(execId: String): Boolean = synchronized { + activeExecutorIds.contains(execId) + } + + // By default, rack is unknown + def getRackForHost(value: String): Option[String] = None + + /** + * Invoked after the system has successfully been initialized. YARN uses this to bootstrap + * allocation of resources based on preferred locations, wait for slave registrations, etc. + */ def postStartHook() { } +} + - // Disconnect from the cluster. - def stop(): Unit +object TaskScheduler { + /** + * 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 - // Submit a sequence of tasks to run. - def submitTasks(taskSet: TaskSet): Unit + // order keyList based on population of value in map + val keyList = _keyList.sortWith( + (left, right) => map(left).size > map(right).size + ) - // Cancel a stage. - def cancelTasks(stageId: Int) + val retval = new ArrayBuffer[T](keyList.size * 2) + var index = 0 + var found = true - // Set the DAG scheduler for upcalls. This is guaranteed to be set before submitTasks is called. - def setDAGScheduler(dagScheduler: DAGScheduler): Unit + while (found) { + found = false + for (key <- keyList) { + val containerList: ArrayBuffer[T] = map.get(key).getOrElse(null) + assert(containerList != null) + // Get the index'th entry for this host - if present + if (index < containerList.size){ + retval += containerList.apply(index) + found = true + } + } + index += 1 + } - // Get the default level of parallelism to use in the cluster, as a hint for sizing jobs. - def defaultParallelism(): Int + retval.toList + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 90f6bcefac..13271b10f3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -17,32 +17,690 @@ package org.apache.spark.scheduler -import java.nio.ByteBuffer +import java.util.Arrays +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet +import scala.math.max +import scala.math.min + +import org.apache.spark.{ExceptionFailure, FetchFailed, Logging, Resubmitted, SparkEnv, + Success, TaskEndReason, TaskKilled, TaskResultLost, TaskState} import org.apache.spark.TaskState.TaskState +import org.apache.spark.scheduler._ +import org.apache.spark.util.{SystemClock, Clock} + /** - * 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). + * Schedules the tasks within a single TaskSet in the TaskScheduler. This class keeps track 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 TaskScheduler - * (e.g. its event handlers). It should not be called from other threads. + * 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 - - def schedulingMode = SchedulingMode.NONE - - def taskSet: TaskSet +private[spark] class TaskSetManager( + sched: TaskScheduler, + val taskSet: TaskSet, + clock: Clock = SystemClock) + extends Schedulable with Logging +{ + // CPUs to request per task + 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 + + // Quantile of tasks at which to start speculation + val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble + val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble + + // Serializer for closures and tasks. + val env = SparkEnv.get + val ser = env.closureSerializer.newInstance() + + val tasks = taskSet.tasks + val numTasks = tasks.length + val copiesRunning = new Array[Int](numTasks) + val successful = new Array[Boolean](numTasks) + val numFailures = new Array[Int](numTasks) + val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) + var tasksSuccessful = 0 + + var weight = 1 + var minShare = 0 + var priority = taskSet.priority + var stageId = taskSet.stageId + var name = "TaskSet_"+taskSet.stageId.toString + var parent: Pool = null + + var runningTasks = 0 + private val runningTasksSet = new HashSet[Long] + + // 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 pendingTasksForExecutor = new HashMap[String, ArrayBuffer[Int]] + + // Set of pending tasks for each host. Similar to pendingTasksForExecutor, + // but at host level. + private val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]] + + // Set of pending tasks for each rack -- similar to the above. + private val pendingTasksForRack = new HashMap[String, ArrayBuffer[Int]] + + // Set containing pending tasks with no locality preferences. + val pendingTasksWithNoPrefs = new ArrayBuffer[Int] + + // 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 + // tasks, we'll just hold them in a HashSet. + val speculatableTasks = new HashSet[Int] + + // Task index, start and finish time for each task attempt (indexed by task ID) + val taskInfos = new HashMap[Long, TaskInfo] + + // Did the 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 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 + val epoch = sched.mapOutputTracker.getEpoch + logDebug("Epoch for " + taskSet + ": " + epoch) + for (t <- tasks) { + t.epoch = epoch + } + + // Add all our tasks to the pending lists. We do this in reverse order + // of task index so that tasks with low indices get launched first. + for (i <- (0 until numTasks).reverse) { + addPendingTask(i) + } + + // Figure out which locality levels we have in our TaskSet, so we can do delay scheduling + val myLocalityLevels = computeValidLocalityLevels() + 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. + // 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 = clock.getTime() // Time we last launched a task at this level + + override def schedulableQueue = null + + override def schedulingMode = SchedulingMode.NONE + + /** + * 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 + } + } + + var hadAliveLocations = false + for (loc <- tasks(index).preferredLocations) { + for (execId <- loc.executorId) { + if (sched.isExecutorAlive(execId)) { + addTo(pendingTasksForExecutor.getOrElseUpdate(execId, new ArrayBuffer)) + hadAliveLocations = true + } + } + 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 + } + } + + 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) + } + + if (!readding) { + allPendingTasks += index // No point scanning this whole list to find the old task there + } + } + + /** + * 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(host: String): ArrayBuffer[Int] = { + pendingTasksForHost.getOrElse(host, ArrayBuffer()) + } + + /** + * 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. + */ + private def findTaskFromList(list: ArrayBuffer[Int]): Option[Int] = { + while (!list.isEmpty) { + val index = list.last + list.trimEnd(1) + if (copiesRunning(index) == 0 && !successful(index)) { + return Some(index) + } + } + return None + } + + /** 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) + } + + /** + * 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 => !successful(index)) // Remove finished tasks from set + + 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)) + } + } + + // 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-local tasks + if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { + 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)) + } + } + } + } + // Check for non-local tasks + if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { + 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 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)) + } + + 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)) { + for { + rack <- sched.getRackForHost(host) + index <- findTaskFromList(getPendingTasksForRack(rack)) + } { + return Some((index, TaskLocality.RACK_LOCAL)) + } + } + + // 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)) { + for (index <- findTaskFromList(allPendingTasks)) { + return Some((index, TaskLocality.ANY)) + } + } + + // Finally, if all else has failed, find a speculative task + return findSpeculativeTask(execId, host, locality) + } + + /** + * Respond to an offer of a single executor from the scheduler by finding a task + */ def resourceOffer( execId: String, host: String, availableCpus: Int, maxLocality: TaskLocality.TaskLocality) - : Option[TaskDescription] + : Option[TaskDescription] = + { + if (tasksSuccessful < numTasks && availableCpus >= CPUS_PER_TASK) { + val curTime = clock.getTime() + + var allowedLocality = getAllowedLocalityLevel(curTime) + if (allowedLocality > maxLocality) { + allowedLocality = maxLocality // We're not allowed to search for farther-away tasks + } + + findTask(execId, host, allowedLocality) 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 + logInfo("Starting task %s:%d as TID %s on executor %s: %s (%s)".format( + taskSet.id, index, taskId, execId, host, taskLocality)) + // Do various bookkeeping + copiesRunning(index) += 1 + val info = new TaskInfo(taskId, index, curTime, execId, host, taskLocality) + taskInfos(taskId) = info + taskAttempts(index) = info :: taskAttempts(index) + // Update our locality level for delay scheduling + currentLocalityIndex = getLocalityIndex(taskLocality) + lastLaunchTime = curTime + // Serialize and return the task + 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 = clock.getTime() - startTime + addRunningTask(taskId) + 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, index, serializedTask)) + } + case _ => + } + } + 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 + } + + private def taskStarted(task: Task[_], info: TaskInfo) { + sched.dagScheduler.taskStarted(task, info) + } + + def handleTaskGettingResult(tid: Long) = { + val info = taskInfos(tid) + info.markGettingResult() + sched.dagScheduler.taskGettingResult(tasks(info.index), info) + } + + /** + * Marks the task as successful and notifies the DAGScheduler that a task has ended. + */ + def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]) = { + val info = taskInfos(tid) + val index = info.index + info.markSuccessful() + removeRunningTask(tid) + if (!successful(index)) { + logInfo("Finished TID %s in %d ms on %s (progress: %d/%d)".format( + tid, info.duration, info.host, tasksSuccessful, numTasks)) + sched.dagScheduler.taskEnded( + tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) + + // Mark successful and stop if all the tasks have succeeded. + tasksSuccessful += 1 + successful(index) = true + if (tasksSuccessful == numTasks) { + sched.taskSetFinished(this) + } + } else { + logInfo("Ignorning task-finished event for TID " + tid + " because task " + + index + " has already completed successfully") + } + } + + /** + * Marks the task as failed, re-adds it to the list of pending tasks, and notifies the + * DAG Scheduler. + */ + def handleFailedTask(tid: Long, state: TaskState, reason: Option[TaskEndReason]) { + val info = taskInfos(tid) + if (info.failed) { + return + } + removeRunningTask(tid) + val index = info.index + info.markFailed() + if (!successful(index)) { + logWarning("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) + copiesRunning(index) -= 1 + // Check if the problem is a map output fetch failure. In that case, this + // task will never succeed on any node, so tell the scheduler about it. + reason.foreach { + case fetchFailed: FetchFailed => + logWarning("Loss was due to fetch failure from " + fetchFailed.bmAddress) + sched.dagScheduler.taskEnded(tasks(index), fetchFailed, null, null, info, null) + successful(index) = true + tasksSuccessful += 1 + sched.taskSetFinished(this) + removeAllRunningTasks() + return + + case TaskKilled => + logWarning("Task %d was killed.".format(tid)) + sched.dagScheduler.taskEnded(tasks(index), reason.get, null, null, info, null) + return + + case ef: ExceptionFailure => + sched.dagScheduler.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null)) + val key = ef.description + val now = clock.getTime() + val (printFull, dupCount) = { + if (recentExceptions.contains(key)) { + val (dupCount, printTime) = recentExceptions(key) + if (now - printTime > EXCEPTION_PRINT_INTERVAL) { + recentExceptions(key) = (0, now) + (true, 0) + } else { + recentExceptions(key) = (dupCount + 1, printTime) + (false, dupCount + 1) + } + } else { + recentExceptions(key) = (0, now) + (true, 0) + } + } + if (printFull) { + val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString)) + logWarning("Loss was due to %s\n%s\n%s".format( + ef.className, ef.description, locs.mkString("\n"))) + } else { + logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount)) + } + + case TaskResultLost => + logWarning("Lost result for TID %s on host %s".format(tid, info.host)) + sched.dagScheduler.taskEnded(tasks(index), TaskResultLost, null, null, info, null) + + case _ => {} + } + // On non-fetch failures, re-enqueue the task as pending for a max number of retries + addPendingTask(index) + if (state != TaskState.KILLED) { + numFailures(index) += 1 + if (numFailures(index) > MAX_TASK_FAILURES) { + logError("Task %s:%d failed more than %d times; aborting job".format( + taskSet.id, index, MAX_TASK_FAILURES)) + abort("Task %s:%d failed more than %d times".format(taskSet.id, index, MAX_TASK_FAILURES)) + } + } + } else { + logInfo("Ignoring task-lost event for TID " + tid + + " because task " + index + " is already finished") + } + } + + def error(message: String) { + // Save the error message + abort("Error: " + message) + } + + def abort(message: String) { + failed = true + causeOfFailure = message + // TODO: Kill running tasks if we were not terminated due to a Mesos error + sched.dagScheduler.taskSetFailed(taskSet, message) + removeAllRunningTasks() + sched.taskSetFinished(this) + } + + /** If the given task ID is not in the set of running tasks, adds it. + * + * Used to keep track of the number of running tasks, for enforcing scheduling policies. + */ + def addRunningTask(tid: Long) { + if (runningTasksSet.add(tid) && parent != null) { + parent.increaseRunningTasks(1) + } + runningTasks = runningTasksSet.size + } + + /** If the given task ID is in the set of running tasks, removes it. */ + def removeRunningTask(tid: Long) { + if (runningTasksSet.remove(tid) && parent != null) { + parent.decreaseRunningTasks(1) + } + runningTasks = runningTasksSet.size + } + + private def removeAllRunningTasks() { + val numRunningTasks = runningTasksSet.size + runningTasksSet.clear() + if (parent != null) { + parent.decreaseRunningTasks(numRunningTasks) + } + runningTasks = 0 + } + + override def getSchedulableByName(name: String): Schedulable = { + return null + } + + override def addSchedulable(schedulable: Schedulable) {} + + override def removeSchedulable(schedulable: Schedulable) {} + + override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + var sortedTaskSetQueue = ArrayBuffer[TaskSetManager](this) + sortedTaskSetQueue += this + return sortedTaskSetQueue + } + + /** Called by TaskScheduler 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) + + // 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 + if (tasks(0).isInstanceOf[ShuffleMapTask]) { + for ((tid, info) <- taskInfos if info.executorId == execId) { + val index = taskInfos(tid).index + if (successful(index)) { + successful(index) = false + copiesRunning(index) -= 1 + tasksSuccessful -= 1 + addPendingTask(index) + // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our + // stage finishes when a total of tasks.size tasks finish. + sched.dagScheduler.taskEnded(tasks(index), Resubmitted, null, null, info, null) + } + } + } + // Also re-enqueue any tasks that were running on the node + for ((tid, info) <- taskInfos if info.running && info.executorId == execId) { + handleFailedTask(tid, TaskState.KILLED, None) + } + } + + /** + * Check for tasks to be speculated and return true if there are any. This is called periodically + * by the TaskScheduler. + * + * TODO: To make this scale to large jobs, we need to maintain a list of running tasks, so that + * we don't scan the whole task set. It might also help to make this sorted by launch time. + */ + override def checkSpeculatableTasks(): Boolean = { + // Can't speculate if we only have one task, or if all tasks have finished. + if (numTasks == 1 || tasksSuccessful == numTasks) { + return false + } + var foundTasks = false + val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt + logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation) + if (tasksSuccessful >= minFinishedForSpeculation && tasksSuccessful > 0) { + val time = clock.getTime() + val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray + Arrays.sort(durations) + val medianDuration = durations(min((0.5 * tasksSuccessful).round.toInt, durations.size - 1)) + val threshold = max(SPECULATION_MULTIPLIER * medianDuration, 100) + // TODO: Threshold should also look at standard deviation of task durations and have a lower + // bound based on that. + logDebug("Task length threshold for speculation: " + threshold) + for ((tid, info) <- taskInfos) { + val index = info.index + if (!successful(index) && copiesRunning(index) == 1 && info.timeRunning(time) > threshold && + !speculatableTasks.contains(index)) { + logInfo( + "Marking task %s:%d (on %s) as speculatable because it ran more than %.0f ms".format( + taskSet.id, index, info.host, threshold)) + speculatableTasks += index + foundTasks = true + } + } + } + return foundTasks + } + + override def hasPendingTasks(): Boolean = { + numTasks > 0 && tasksSuccessful < 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 + } + } - def error(message: String) + /** + * 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 + logDebug("Valid locality levels for " + taskSet + ": " + levels.mkString(", ")) + levels.toArray + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala new file mode 100644 index 0000000000..ba6bab3f91 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 + +/** + * Represents free resources available on an executor. + */ +private[spark] +class WorkerOffer(val executorId: String, val host: String, val cores: Int) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala deleted file mode 100644 index 85033958ef..0000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala +++ /dev/null @@ -1,486 +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.spark.scheduler.cluster - -import java.nio.ByteBuffer -import java.util.concurrent.atomic.AtomicLong -import java.util.{TimerTask, Timer} - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet - -import org.apache.spark._ -import org.apache.spark.TaskState.TaskState -import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.SchedulingMode.SchedulingMode - -/** - * The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call - * 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 - 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 - - // ClusterTaskSetManagers are not thread safe, so any access to one should be synchronized - // on this class. - val activeTaskSets = new HashMap[String, ClusterTaskSetManager] - - val taskIdToTaskSetId = new HashMap[Long, String] - val taskIdToExecutorId = new HashMap[Long, String] - val taskSetTaskIds = new HashMap[String, HashSet[Long]] - - @volatile private var hasReceivedTask = false - @volatile private var hasLaunchedTask = false - private val starvationTimer = new Timer(true) - - // Incrementing task IDs - val nextTaskId = new AtomicLong(0) - - // Which executor IDs we have executors on - val activeExecutorIds = new 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 executorsByHost = new HashMap[String, HashSet[String]] - - private val executorIdToHost = new HashMap[String, String] - - // Listener object to pass upcalls into - var dagScheduler: DAGScheduler = null - - var backend: SchedulerBackend = null - - val mapOutputTracker = SparkEnv.get.mapOutputTracker - - var schedulableBuilder: SchedulableBuilder = null - var rootPool: Pool = null - // default scheduler is FIFO - val schedulingMode: SchedulingMode = SchedulingMode.withName( - System.getProperty("spark.scheduler.mode", "FIFO")) - - // This is a var so that we can reset it for testing purposes. - private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) - - override def setDAGScheduler(dagScheduler: DAGScheduler) { - this.dagScheduler = dagScheduler - } - - def initialize(context: SchedulerBackend) { - backend = context - // temporarily set rootPool name to empty - rootPool = new Pool("", schedulingMode, 0, 0) - schedulableBuilder = { - schedulingMode match { - case SchedulingMode.FIFO => - new FIFOSchedulableBuilder(rootPool) - case SchedulingMode.FAIR => - new FairSchedulableBuilder(rootPool) - } - } - schedulableBuilder.buildPools() - } - - def newTaskId(): Long = nextTaskId.getAndIncrement() - - override def start() { - backend.start() - - if (System.getProperty("spark.speculation", "false").toBoolean) { - new Thread("ClusterScheduler speculation check") { - setDaemon(true) - - override def run() { - logInfo("Starting speculative execution thread") - while (true) { - try { - Thread.sleep(SPECULATION_INTERVAL) - } catch { - case e: InterruptedException => {} - } - checkSpeculatableTasks() - } - } - }.start() - } - } - - override def submitTasks(taskSet: TaskSet) { - val tasks = taskSet.tasks - logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks") - this.synchronized { - val manager = new ClusterTaskSetManager(this, taskSet) - activeTaskSets(taskSet.id) = manager - schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) - taskSetTaskIds(taskSet.id) = new HashSet[Long]() - - if (!hasReceivedTask) { - starvationTimer.scheduleAtFixedRate(new TimerTask() { - override def run() { - if (!hasLaunchedTask) { - logWarning("Initial job has not accepted any resources; " + - "check your cluster UI to ensure that workers are registered " + - "and have sufficient memory") - } else { - this.cancel() - } - } - }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) - } - hasReceivedTask = true - } - backend.reviveOffers() - } - - override def cancelTasks(stageId: Int): Unit = synchronized { - logInfo("Cancelling stage " + stageId) - activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) => - // There are two possible cases here: - // 1. The task set manager has been created and some tasks have been scheduled. - // In this case, send a kill signal to the executors to kill the task and then abort - // the stage. - // 2. The task set manager has been created but no tasks has been scheduled. In this case, - // simply abort the stage. - val taskIds = taskSetTaskIds(tsm.taskSet.id) - if (taskIds.size > 0) { - taskIds.foreach { tid => - val execId = taskIdToExecutorId(tid) - backend.killTask(tid, execId) - } - } - tsm.error("Stage %d was cancelled".format(stageId)) - } - } - - def taskSetFinished(manager: TaskSetManager): Unit = synchronized { - // Check to see if the given task set has been removed. This is possible in the case of - // multiple unrecoverable task failures (e.g. if the entire task set is killed when it has - // more than one running tasks). - if (activeTaskSets.contains(manager.taskSet.id)) { - activeTaskSets -= manager.taskSet.id - manager.parent.removeSchedulable(manager) - logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) - taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) - taskIdToExecutorId --= taskSetTaskIds(manager.taskSet.id) - taskSetTaskIds.remove(manager.taskSet.id) - } - } - - /** - * Called by cluster manager to offer resources on slaves. We respond by asking our active task - * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so - * that tasks are balanced across the cluster. - */ - def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = synchronized { - 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) - } - } - - // 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 sortedTaskSets = rootPool.getSortedTaskSetQueue() - for (taskSet <- sortedTaskSets) { - logDebug("parentName: %s, name: %s, runningTasks: %s".format( - 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 (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 <- taskSet.resourceOffer(execId, host, availableCpus(i), maxLocality)) { - tasks(i) += task - val tid = task.taskId - taskIdToTaskSetId(tid) = taskSet.taskSet.id - taskSetTaskIds(taskSet.taskSet.id) += tid - taskIdToExecutorId(tid) = execId - activeExecutorIds += execId - executorsByHost(host) += execId - availableCpus(i) -= 1 - launchedTask = true - } - } - } while (launchedTask) - } - - if (tasks.size > 0) { - hasLaunchedTask = true - } - return tasks - } - - def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { - var failedExecutor: Option[String] = None - var taskFailed = false - synchronized { - try { - if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) { - // We lost this entire executor, so remember that it's gone - val execId = taskIdToExecutorId(tid) - if (activeExecutorIds.contains(execId)) { - removeExecutor(execId) - failedExecutor = Some(execId) - } - } - taskIdToTaskSetId.get(tid) match { - case Some(taskSetId) => - if (TaskState.isFinished(state)) { - taskIdToTaskSetId.remove(tid) - if (taskSetTaskIds.contains(taskSetId)) { - taskSetTaskIds(taskSetId) -= tid - } - taskIdToExecutorId.remove(tid) - } - if (state == TaskState.FAILED) { - taskFailed = true - } - activeTaskSets.get(taskSetId).foreach { taskSet => - if (state == TaskState.FINISHED) { - taskSet.removeRunningTask(tid) - taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) - } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { - taskSet.removeRunningTask(tid) - taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData) - } - } - case None => - logInfo("Ignoring update from TID " + tid + " because its task set is gone") - } - } catch { - case e: Exception => logError("Exception in statusUpdate", e) - } - } - // Update the DAGScheduler without holding a lock on this, since that can deadlock - if (failedExecutor != None) { - dagScheduler.executorLost(failedExecutor.get) - backend.reviveOffers() - } - if (taskFailed) { - // Also revive offers if a task had failed for some reason other than host lost - backend.reviveOffers() - } - } - - def handleTaskGettingResult(taskSetManager: ClusterTaskSetManager, tid: Long) { - taskSetManager.handleTaskGettingResult(tid) - } - - def handleSuccessfulTask( - taskSetManager: ClusterTaskSetManager, - tid: Long, - taskResult: DirectTaskResult[_]) = synchronized { - taskSetManager.handleSuccessfulTask(tid, taskResult) - } - - def handleFailedTask( - taskSetManager: ClusterTaskSetManager, - tid: Long, - taskState: TaskState, - reason: Option[TaskEndReason]) = synchronized { - taskSetManager.handleFailedTask(tid, taskState, reason) - if (taskState == TaskState.FINISHED) { - // The task finished successfully but the result was lost, so we should revive offers. - backend.reviveOffers() - } - } - - def error(message: String) { - synchronized { - if (activeTaskSets.size > 0) { - // Have each task set throw a SparkException with the error - for ((taskSetId, manager) <- activeTaskSets) { - try { - manager.error(message) - } catch { - case e: Exception => logError("Exception in error callback", e) - } - } - } else { - // No task sets are active but we still got an error. Just exit since this - // must mean the error is during registration. - // It might be good to do something smarter here in the future. - logError("Exiting due to error from cluster scheduler: " + message) - System.exit(1) - } - } - } - - override def stop() { - if (backend != null) { - backend.stop() - } - if (taskResultGetter != null) { - taskResultGetter.stop() - } - - // sleeping for an arbitrary 5 seconds : to ensure that messages are sent out. - // TODO: Do something better ! - Thread.sleep(5000L) - } - - override def defaultParallelism() = backend.defaultParallelism() - - - // Check for speculatable tasks in all our active jobs. - def checkSpeculatableTasks() { - var shouldRevive = false - synchronized { - shouldRevive = rootPool.checkSpeculatableTasks() - } - if (shouldRevive) { - backend.reviveOffers() - } - } - - // Check for pending tasks in all our active jobs. - def hasPendingTasks: Boolean = { - synchronized { - rootPool.hasPendingTasks() - } - } - - def executorLost(executorId: String, reason: ExecutorLossReason) { - var failedExecutor: Option[String] = None - - synchronized { - if (activeExecutorIds.contains(executorId)) { - val hostPort = executorIdToHost(executorId) - logError("Lost executor %s on %s: %s".format(executorId, hostPort, reason)) - removeExecutor(executorId) - failedExecutor = Some(executorId) - } else { - // We may get multiple executorLost() calls with different loss reasons. For example, one - // may be triggered by a dropped connection from the slave while another may be a report - // of executor termination from Mesos. We produce log messages for both so we eventually - // report the termination reason. - logError("Lost an executor " + executorId + " (already removed): " + reason) - } - } - // Call dagScheduler.executorLost without holding the lock on this to prevent deadlock - if (failedExecutor != None) { - dagScheduler.executorLost(failedExecutor.get) - backend.reviveOffers() - } - } - - /** Remove an executor from all our data structures and mark it as lost */ - private def removeExecutor(executorId: String) { - activeExecutorIds -= executorId - val host = executorIdToHost(executorId) - val execs = executorsByHost.getOrElse(host, new HashSet) - execs -= executorId - if (execs.isEmpty) { - executorsByHost -= host - } - executorIdToHost -= executorId - rootPool.executorLost(executorId, host) - } - - def executorGained(execId: String, host: String) { - dagScheduler.executorGained(execId, host) - } - - def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized { - executorsByHost.get(host).map(_.toSet) - } - - def hasExecutorsAliveOnHost(host: String): Boolean = synchronized { - executorsByHost.contains(host) - } - - def isExecutorAlive(execId: String): Boolean = synchronized { - activeExecutorIds.contains(execId) - } - - // By default, rack is unknown - def getRackForHost(value: String): Option[String] = None -} - - -object ClusterScheduler { - /** - * 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 - - // order keyList based on population of value in map - val keyList = _keyList.sortWith( - (left, right) => map(left).size > map(right).size - ) - - val retval = new ArrayBuffer[T](keyList.size * 2) - var index = 0 - var found = true - - while (found) { - found = false - for (key <- keyList) { - val containerList: ArrayBuffer[T] = map.get(key).getOrElse(null) - assert(containerList != null) - // Get the index'th entry for this host - if present - if (index < containerList.size){ - retval += containerList.apply(index) - found = true - } - } - index += 1 - } - - retval.toList - } -} 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 deleted file mode 100644 index ee47aaffca..0000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ /dev/null @@ -1,703 +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.spark.scheduler.cluster - -import java.util.Arrays - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import scala.math.max -import scala.math.min - -import org.apache.spark.{ExceptionFailure, FetchFailed, Logging, Resubmitted, SparkEnv, - Success, TaskEndReason, TaskKilled, TaskResultLost, TaskState} -import org.apache.spark.TaskState.TaskState -import org.apache.spark.scheduler._ -import org.apache.spark.util.{SystemClock, Clock} - - -/** - * 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, - clock: Clock = SystemClock) - extends TaskSetManager - with Logging -{ - // CPUs to request per task - 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 - - // Quantile of tasks at which to start speculation - val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble - val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble - - // Serializer for closures and tasks. - val env = SparkEnv.get - val ser = env.closureSerializer.newInstance() - - val tasks = taskSet.tasks - val numTasks = tasks.length - val copiesRunning = new Array[Int](numTasks) - val successful = new Array[Boolean](numTasks) - val numFailures = new Array[Int](numTasks) - val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) - var tasksSuccessful = 0 - - var weight = 1 - var minShare = 0 - var priority = taskSet.priority - var stageId = taskSet.stageId - var name = "TaskSet_"+taskSet.stageId.toString - var parent: Pool = null - - var runningTasks = 0 - private val runningTasksSet = new HashSet[Long] - - // 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 pendingTasksForExecutor = new HashMap[String, ArrayBuffer[Int]] - - // Set of pending tasks for each host. Similar to pendingTasksForExecutor, - // but at host level. - private val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]] - - // Set of pending tasks for each rack -- similar to the above. - private val pendingTasksForRack = new HashMap[String, ArrayBuffer[Int]] - - // Set containing pending tasks with no locality preferences. - val pendingTasksWithNoPrefs = new ArrayBuffer[Int] - - // 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 - // tasks, we'll just hold them in a HashSet. - val speculatableTasks = new HashSet[Int] - - // Task index, start and finish time for each task attempt (indexed by task ID) - val taskInfos = new HashMap[Long, TaskInfo] - - // Did the 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 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 - val epoch = sched.mapOutputTracker.getEpoch - logDebug("Epoch for " + taskSet + ": " + epoch) - for (t <- tasks) { - t.epoch = epoch - } - - // Add all our tasks to the pending lists. We do this in reverse order - // of task index so that tasks with low indices get launched first. - for (i <- (0 until numTasks).reverse) { - addPendingTask(i) - } - - // Figure out which locality levels we have in our TaskSet, so we can do delay scheduling - val myLocalityLevels = computeValidLocalityLevels() - 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. - // 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 = 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 - * 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 - } - } - - var hadAliveLocations = false - for (loc <- tasks(index).preferredLocations) { - for (execId <- loc.executorId) { - if (sched.isExecutorAlive(execId)) { - addTo(pendingTasksForExecutor.getOrElseUpdate(execId, new ArrayBuffer)) - hadAliveLocations = true - } - } - 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 - } - } - - 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) - } - - if (!readding) { - allPendingTasks += index // No point scanning this whole list to find the old task there - } - } - - /** - * 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(host: String): ArrayBuffer[Int] = { - pendingTasksForHost.getOrElse(host, ArrayBuffer()) - } - - /** - * 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. - */ - private def findTaskFromList(list: ArrayBuffer[Int]): Option[Int] = { - while (!list.isEmpty) { - val index = list.last - list.trimEnd(1) - if (copiesRunning(index) == 0 && !successful(index)) { - return Some(index) - } - } - return None - } - - /** 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) - } - - /** - * 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 => !successful(index)) // Remove finished tasks from set - - 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)) - } - } - - // 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-local tasks - if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { - 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)) - } - } - } - } - - // Check for non-local tasks - if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { - 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 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)) - } - - 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)) { - for { - rack <- sched.getRackForHost(host) - index <- findTaskFromList(getPendingTasksForRack(rack)) - } { - return Some((index, TaskLocality.RACK_LOCAL)) - } - } - - // 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)) { - for (index <- findTaskFromList(allPendingTasks)) { - return Some((index, TaskLocality.ANY)) - } - } - - // Finally, if all else has failed, find a speculative task - return findSpeculativeTask(execId, host, locality) - } - - /** - * Respond to an offer of a single executor from the scheduler by finding a task - */ - override def resourceOffer( - execId: String, - host: String, - availableCpus: Int, - maxLocality: TaskLocality.TaskLocality) - : Option[TaskDescription] = - { - if (tasksSuccessful < numTasks && availableCpus >= CPUS_PER_TASK) { - val curTime = clock.getTime() - - var allowedLocality = getAllowedLocalityLevel(curTime) - if (allowedLocality > maxLocality) { - allowedLocality = maxLocality // We're not allowed to search for farther-away tasks - } - - findTask(execId, host, allowedLocality) 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 - logInfo("Starting task %s:%d as TID %s on executor %s: %s (%s)".format( - taskSet.id, index, taskId, execId, host, taskLocality)) - // Do various bookkeeping - copiesRunning(index) += 1 - val info = new TaskInfo(taskId, index, curTime, execId, host, taskLocality) - taskInfos(taskId) = info - taskAttempts(index) = info :: taskAttempts(index) - // Update our locality level for delay scheduling - currentLocalityIndex = getLocalityIndex(taskLocality) - lastLaunchTime = curTime - // Serialize and return the task - 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 = clock.getTime() - startTime - addRunningTask(taskId) - 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, index, serializedTask)) - } - case _ => - } - } - 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 - } - - private def taskStarted(task: Task[_], info: TaskInfo) { - sched.dagScheduler.taskStarted(task, info) - } - - def handleTaskGettingResult(tid: Long) = { - val info = taskInfos(tid) - info.markGettingResult() - sched.dagScheduler.taskGettingResult(tasks(info.index), info) - } - - /** - * Marks the task as successful and notifies the DAGScheduler that a task has ended. - */ - def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]) = { - val info = taskInfos(tid) - val index = info.index - info.markSuccessful() - removeRunningTask(tid) - if (!successful(index)) { - logInfo("Finished TID %s in %d ms on %s (progress: %d/%d)".format( - tid, info.duration, info.host, tasksSuccessful, numTasks)) - sched.dagScheduler.taskEnded( - tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) - - // Mark successful and stop if all the tasks have succeeded. - tasksSuccessful += 1 - successful(index) = true - if (tasksSuccessful == numTasks) { - sched.taskSetFinished(this) - } - } else { - logInfo("Ignorning task-finished event for TID " + tid + " because task " + - index + " has already completed successfully") - } - } - - /** - * Marks the task as failed, re-adds it to the list of pending tasks, and notifies the - * DAG Scheduler. - */ - def handleFailedTask(tid: Long, state: TaskState, reason: Option[TaskEndReason]) { - val info = taskInfos(tid) - if (info.failed) { - return - } - removeRunningTask(tid) - val index = info.index - info.markFailed() - if (!successful(index)) { - logWarning("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) - copiesRunning(index) -= 1 - // Check if the problem is a map output fetch failure. In that case, this - // task will never succeed on any node, so tell the scheduler about it. - reason.foreach { - case fetchFailed: FetchFailed => - logWarning("Loss was due to fetch failure from " + fetchFailed.bmAddress) - sched.dagScheduler.taskEnded(tasks(index), fetchFailed, null, null, info, null) - successful(index) = true - tasksSuccessful += 1 - sched.taskSetFinished(this) - removeAllRunningTasks() - return - - case TaskKilled => - logWarning("Task %d was killed.".format(tid)) - sched.dagScheduler.taskEnded(tasks(index), reason.get, null, null, info, null) - return - - case ef: ExceptionFailure => - sched.dagScheduler.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null)) - val key = ef.description - val now = clock.getTime() - val (printFull, dupCount) = { - if (recentExceptions.contains(key)) { - val (dupCount, printTime) = recentExceptions(key) - if (now - printTime > EXCEPTION_PRINT_INTERVAL) { - recentExceptions(key) = (0, now) - (true, 0) - } else { - recentExceptions(key) = (dupCount + 1, printTime) - (false, dupCount + 1) - } - } else { - recentExceptions(key) = (0, now) - (true, 0) - } - } - if (printFull) { - val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString)) - logWarning("Loss was due to %s\n%s\n%s".format( - ef.className, ef.description, locs.mkString("\n"))) - } else { - logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount)) - } - - case TaskResultLost => - logWarning("Lost result for TID %s on host %s".format(tid, info.host)) - sched.dagScheduler.taskEnded(tasks(index), TaskResultLost, null, null, info, null) - - case _ => {} - } - // On non-fetch failures, re-enqueue the task as pending for a max number of retries - addPendingTask(index) - if (state != TaskState.KILLED) { - numFailures(index) += 1 - if (numFailures(index) > MAX_TASK_FAILURES) { - logError("Task %s:%d failed more than %d times; aborting job".format( - taskSet.id, index, MAX_TASK_FAILURES)) - abort("Task %s:%d failed more than %d times".format(taskSet.id, index, MAX_TASK_FAILURES)) - } - } - } else { - logInfo("Ignoring task-lost event for TID " + tid + - " because task " + index + " is already finished") - } - } - - override def error(message: String) { - // Save the error message - abort("Error: " + message) - } - - def abort(message: String) { - failed = true - causeOfFailure = message - // TODO: Kill running tasks if we were not terminated due to a Mesos error - sched.dagScheduler.taskSetFailed(taskSet, message) - removeAllRunningTasks() - sched.taskSetFinished(this) - } - - /** If the given task ID is not in the set of running tasks, adds it. - * - * Used to keep track of the number of running tasks, for enforcing scheduling policies. - */ - def addRunningTask(tid: Long) { - if (runningTasksSet.add(tid) && parent != null) { - parent.increaseRunningTasks(1) - } - runningTasks = runningTasksSet.size - } - - /** If the given task ID is in the set of running tasks, removes it. */ - def removeRunningTask(tid: Long) { - if (runningTasksSet.remove(tid) && parent != null) { - parent.decreaseRunningTasks(1) - } - runningTasks = runningTasksSet.size - } - - private def removeAllRunningTasks() { - val numRunningTasks = runningTasksSet.size - runningTasksSet.clear() - if (parent != null) { - parent.decreaseRunningTasks(numRunningTasks) - } - runningTasks = 0 - } - - override def getSchedulableByName(name: String): Schedulable = { - return null - } - - override def addSchedulable(schedulable: Schedulable) {} - - override def removeSchedulable(schedulable: Schedulable) {} - - override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { - var sortedTaskSetQueue = ArrayBuffer[TaskSetManager](this) - sortedTaskSetQueue += this - return sortedTaskSetQueue - } - - /** 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) - - // 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 - if (tasks(0).isInstanceOf[ShuffleMapTask]) { - for ((tid, info) <- taskInfos if info.executorId == execId) { - val index = taskInfos(tid).index - if (successful(index)) { - successful(index) = false - copiesRunning(index) -= 1 - tasksSuccessful -= 1 - addPendingTask(index) - // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our - // stage finishes when a total of tasks.size tasks finish. - sched.dagScheduler.taskEnded(tasks(index), Resubmitted, null, null, info, null) - } - } - } - // Also re-enqueue any tasks that were running on the node - for ((tid, info) <- taskInfos if info.running && info.executorId == execId) { - handleFailedTask(tid, TaskState.KILLED, None) - } - } - - /** - * Check for tasks to be speculated and return true if there are any. This is called periodically - * by the ClusterScheduler. - * - * TODO: To make this scale to large jobs, we need to maintain a list of running tasks, so that - * we don't scan the whole task set. It might also help to make this sorted by launch time. - */ - override def checkSpeculatableTasks(): Boolean = { - // Can't speculate if we only have one task, or if all tasks have finished. - if (numTasks == 1 || tasksSuccessful == numTasks) { - return false - } - var foundTasks = false - val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt - logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation) - if (tasksSuccessful >= minFinishedForSpeculation && tasksSuccessful > 0) { - val time = clock.getTime() - val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray - Arrays.sort(durations) - val medianDuration = durations(min((0.5 * tasksSuccessful).round.toInt, durations.size - 1)) - val threshold = max(SPECULATION_MULTIPLIER * medianDuration, 100) - // TODO: Threshold should also look at standard deviation of task durations and have a lower - // bound based on that. - logDebug("Task length threshold for speculation: " + threshold) - for ((tid, info) <- taskInfos) { - val index = info.index - if (!successful(index) && copiesRunning(index) == 1 && info.timeRunning(time) > threshold && - !speculatableTasks.contains(index)) { - logInfo( - "Marking task %s:%d (on %s) as speculatable because it ran more than %.0f ms".format( - taskSet.id, index, info.host, threshold)) - speculatableTasks += index - foundTasks = true - } - } - } - return foundTasks - } - - override def hasPendingTasks(): Boolean = { - numTasks > 0 && tasksSuccessful < 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 - logDebug("Valid locality levels for " + taskSet + ": " + levels.mkString(", ")) - levels.toArray - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 70f3f88401..b8ac498527 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -29,7 +29,8 @@ import akka.util.Duration import akka.util.duration._ import org.apache.spark.{SparkException, Logging, TaskState} -import org.apache.spark.scheduler.TaskDescription +import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, TaskScheduler, + WorkerOffer} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.Utils @@ -42,7 +43,7 @@ import org.apache.spark.util.Utils * (spark.deploy.*). */ private[spark] -class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: ActorSystem) +class CoarseGrainedSchedulerBackend(scheduler: TaskScheduler, actorSystem: ActorSystem) extends SchedulerBackend with Logging { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorLossReason.scala deleted file mode 100644 index 5077b2b48b..0000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorLossReason.scala +++ /dev/null @@ -1,38 +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.spark.scheduler.cluster - -import org.apache.spark.executor.ExecutorExitCode - -/** - * Represents an explanation for a executor or whole slave failing or exiting. - */ -private[spark] -class ExecutorLossReason(val message: String) { - override def toString: String = message -} - -private[spark] -case class ExecutorExited(val exitCode: Int) - extends ExecutorLossReason(ExecutorExitCode.explainExitCode(exitCode)) { -} - -private[spark] -case class SlaveLost(_message: String = "Slave lost") - extends ExecutorLossReason(_message) { -} 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 deleted file mode 100644 index 5367218faa..0000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala +++ /dev/null @@ -1,37 +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.spark.scheduler.cluster - -import org.apache.spark.SparkContext - -/** - * A backend interface for cluster scheduling systems that allows plugging in different ones under - * ClusterScheduler. We assume a Mesos-like model where the application gets resource offers as - * machines become available and can launch tasks on them. - */ -private[spark] trait SchedulerBackend { - def start(): Unit - def stop(): Unit - def reviveOffers(): Unit - def defaultParallelism(): Int - - def killTask(taskId: Long, executorId: String): Unit = throw new UnsupportedOperationException - - // Memory used by each executor (in megabytes) - protected val executorMemory: Int = SparkContext.executorMemoryRequested -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index d78bdbaa7a..a589e7456f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -19,10 +19,12 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem} + import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.scheduler.TaskScheduler private[spark] class SimrSchedulerBackend( - scheduler: ClusterScheduler, + scheduler: TaskScheduler, sc: SparkContext, driverFilePath: String) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) 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 cefa970bb9..15c600a1ec 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,14 +17,16 @@ package org.apache.spark.scheduler.cluster +import scala.collection.mutable.HashMap + 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.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskScheduler} import org.apache.spark.util.Utils private[spark] class SparkDeploySchedulerBackend( - scheduler: ClusterScheduler, + scheduler: TaskScheduler, sc: SparkContext, masters: Array[String], appName: String) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala deleted file mode 100644 index 2064d97b49..0000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala +++ /dev/null @@ -1,108 +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.spark.scheduler.cluster - -import java.nio.ByteBuffer -import java.util.concurrent.{LinkedBlockingDeque, ThreadFactory, ThreadPoolExecutor, TimeUnit} - -import org.apache.spark._ -import org.apache.spark.TaskState.TaskState -import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, TaskResult} -import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.Utils - -/** - * Runs a thread pool that deserializes and remotely fetches (if necessary) task results. - */ -private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler) - extends Logging { - private val THREADS = System.getProperty("spark.resultGetter.threads", "4").toInt - private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( - THREADS, "Result resolver thread") - - protected val serializer = new ThreadLocal[SerializerInstance] { - override def initialValue(): SerializerInstance = { - return sparkEnv.closureSerializer.newInstance() - } - } - - def enqueueSuccessfulTask( - taskSetManager: ClusterTaskSetManager, tid: Long, serializedData: ByteBuffer) { - getTaskResultExecutor.execute(new Runnable { - override def run() { - try { - val result = serializer.get().deserialize[TaskResult[_]](serializedData) match { - case directResult: DirectTaskResult[_] => directResult - case IndirectTaskResult(blockId) => - logDebug("Fetching indirect task result for TID %s".format(tid)) - scheduler.handleTaskGettingResult(taskSetManager, tid) - val serializedTaskResult = sparkEnv.blockManager.getRemoteBytes(blockId) - if (!serializedTaskResult.isDefined) { - /* We won't be able to get the task result if the machine that ran the task failed - * between when the task ended and when we tried to fetch the result, or if the - * block manager had to flush the result. */ - scheduler.handleFailedTask( - taskSetManager, tid, TaskState.FINISHED, Some(TaskResultLost)) - return - } - val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]]( - serializedTaskResult.get) - sparkEnv.blockManager.master.removeBlock(blockId) - deserializedResult - } - result.metrics.resultSize = serializedData.limit() - scheduler.handleSuccessfulTask(taskSetManager, tid, result) - } catch { - case cnf: ClassNotFoundException => - val loader = Thread.currentThread.getContextClassLoader - taskSetManager.abort("ClassNotFound with classloader: " + loader) - case ex => - taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex)) - } - } - }) - } - - def enqueueFailedTask(taskSetManager: ClusterTaskSetManager, tid: Long, taskState: TaskState, - serializedData: ByteBuffer) { - var reason: Option[TaskEndReason] = None - getTaskResultExecutor.execute(new Runnable { - override def run() { - try { - if (serializedData != null && serializedData.limit() > 0) { - reason = Some(serializer.get().deserialize[TaskEndReason]( - serializedData, getClass.getClassLoader)) - } - } catch { - case cnd: ClassNotFoundException => - // Log an error but keep going here -- the task failed, so not catastropic if we can't - // deserialize the reason. - val loader = Thread.currentThread.getContextClassLoader - logError( - "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) - case ex => {} - } - scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) - } - }) - } - - def stop() { - getTaskResultExecutor.shutdownNow() - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/WorkerOffer.scala deleted file mode 100644 index 938f62883a..0000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/WorkerOffer.scala +++ /dev/null @@ -1,24 +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.spark.scheduler.cluster - -/** - * Represents free resources available on an executor. - */ -private[spark] -class WorkerOffer(val executorId: String, val host: String, val cores: Int) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 300fe693f1..310da0027e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -30,7 +30,8 @@ import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} import org.apache.spark.{SparkException, Logging, SparkContext, TaskState} -import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend} +import org.apache.spark.scheduler.TaskScheduler +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend /** * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds @@ -43,7 +44,7 @@ import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedu * remove this. */ private[spark] class CoarseMesosSchedulerBackend( - scheduler: ClusterScheduler, + scheduler: TaskScheduler, sc: SparkContext, master: String, appName: String) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 50cbc2ca92..c0e99df0b6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -30,9 +30,8 @@ import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} import org.apache.spark.{Logging, SparkException, SparkContext, TaskState} -import org.apache.spark.scheduler.TaskDescription -import org.apache.spark.scheduler.cluster.{ClusterScheduler, ExecutorExited, ExecutorLossReason} -import org.apache.spark.scheduler.cluster.{SchedulerBackend, SlaveLost, WorkerOffer} +import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SchedulerBackend, SlaveLost, + TaskDescription, TaskScheduler, WorkerOffer} import org.apache.spark.util.Utils /** @@ -41,7 +40,7 @@ import org.apache.spark.util.Utils * from multiple apps can run on different cores) and in time (a core can switch ownership). */ private[spark] class MesosSchedulerBackend( - scheduler: ClusterScheduler, + scheduler: TaskScheduler, sc: SparkContext, master: String, appName: String) @@ -210,7 +209,7 @@ private[spark] class MesosSchedulerBackend( getResource(offer.getResourcesList, "cpus").toInt) } - // Call into the ClusterScheduler + // Call into the TaskScheduler val taskLists = scheduler.resourceOffers(offerableWorkers) // Build a list of Mesos tasks for each slave diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala new file mode 100644 index 0000000000..96c3a03602 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.local + +import java.nio.ByteBuffer + +import akka.actor.{Actor, ActorRef, Props} + +import org.apache.spark.{SparkContext, SparkEnv, TaskState} +import org.apache.spark.TaskState.TaskState +import org.apache.spark.executor.{Executor, ExecutorBackend} +import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, WorkerOffer} + +/** + * LocalBackend sits behind a TaskScheduler and handles launching tasks on a single Executor + * (created by the LocalBackend) running locally. + * + * THREADING: Because methods can be called both from the Executor and the TaskScheduler, and + * because the Executor class is not thread safe, all methods are synchronized. + */ +private[spark] class LocalBackend(scheduler: TaskScheduler, private val totalCores: Int) + extends SchedulerBackend with ExecutorBackend { + + private var freeCores = totalCores + + private val localExecutorId = "localhost" + private val localExecutorHostname = "localhost" + + val executor = new Executor(localExecutorId, localExecutorHostname, Seq.empty, isLocal = true) + + override def start() { + } + + override def stop() { + } + + override def reviveOffers() = synchronized { + val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) + for (task <- scheduler.resourceOffers(offers).flatten) { + freeCores -= 1 + executor.launchTask(this, task.taskId, task.serializedTask) + } + } + + override def defaultParallelism() = totalCores + + override def killTask(taskId: Long, executorId: String) = synchronized { + executor.killTask(taskId) + } + + override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) = synchronized { + scheduler.statusUpdate(taskId, state, serializedData) + if (TaskState.isFinished(state)) { + freeCores += 1 + reviveOffers() + } + } +} 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 deleted file mode 100644 index 2699f0b33e..0000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala +++ /dev/null @@ -1,219 +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.spark.scheduler.local - -import java.nio.ByteBuffer -import java.util.concurrent.atomic.AtomicInteger - -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} - -import akka.actor._ - -import org.apache.spark._ -import org.apache.spark.TaskState.TaskState -import org.apache.spark.executor.{Executor, ExecutorBackend} -import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.SchedulingMode.SchedulingMode - - -/** - * A FIFO or Fair TaskScheduler implementation that runs tasks locally in a thread pool. Optionally - * the scheduler also allows each task to fail up to maxFailures times, which is useful for - * testing fault recovery. - */ - -private[local] -case class LocalReviveOffers() - -private[local] -case class LocalStatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) - -private[local] -case class KillTask(taskId: Long) - -private[spark] -class LocalActor(localScheduler: LocalScheduler, private var freeCores: Int) - extends Actor with Logging { - - val executor = new Executor("localhost", "localhost", Seq.empty, isLocal = true) - - def receive = { - case LocalReviveOffers => - launchTask(localScheduler.resourceOffer(freeCores)) - - case LocalStatusUpdate(taskId, state, serializeData) => - if (TaskState.isFinished(state)) { - freeCores += 1 - launchTask(localScheduler.resourceOffer(freeCores)) - } - - case KillTask(taskId) => - executor.killTask(taskId) - } - - private def launchTask(tasks: Seq[TaskDescription]) { - for (task <- tasks) { - freeCores -= 1 - executor.launchTask(localScheduler, task.taskId, task.serializedTask) - } - } -} - -private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: SparkContext) - extends TaskScheduler - with ExecutorBackend - with Logging { - - val env = SparkEnv.get - val attemptId = new AtomicInteger - var dagScheduler: DAGScheduler = null - - // 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. - val currentFiles: HashMap[String, Long] = new HashMap[String, Long]() - val currentJars: HashMap[String, Long] = new HashMap[String, Long]() - - var schedulableBuilder: SchedulableBuilder = null - var rootPool: Pool = null - val schedulingMode: SchedulingMode = SchedulingMode.withName( - System.getProperty("spark.scheduler.mode", "FIFO")) - val activeTaskSets = new HashMap[String, LocalTaskSetManager] - val taskIdToTaskSetId = new HashMap[Long, String] - val taskSetTaskIds = new HashMap[String, HashSet[Long]] - - var localActor: ActorRef = null - - override def start() { - // temporarily set rootPool name to empty - rootPool = new Pool("", schedulingMode, 0, 0) - schedulableBuilder = { - schedulingMode match { - case SchedulingMode.FIFO => - new FIFOSchedulableBuilder(rootPool) - case SchedulingMode.FAIR => - new FairSchedulableBuilder(rootPool) - } - } - schedulableBuilder.buildPools() - - localActor = env.actorSystem.actorOf(Props(new LocalActor(this, threads)), "Test") - } - - override def setDAGScheduler(dagScheduler: DAGScheduler) { - this.dagScheduler = dagScheduler - } - - override def submitTasks(taskSet: TaskSet) { - synchronized { - val manager = new LocalTaskSetManager(this, taskSet) - schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) - activeTaskSets(taskSet.id) = manager - taskSetTaskIds(taskSet.id) = new HashSet[Long]() - localActor ! LocalReviveOffers - } - } - - override def cancelTasks(stageId: Int): Unit = synchronized { - logInfo("Cancelling stage " + stageId) - logInfo("Cancelling stage " + activeTaskSets.map(_._2.stageId)) - activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) => - // There are two possible cases here: - // 1. The task set manager has been created and some tasks have been scheduled. - // In this case, send a kill signal to the executors to kill the task and then abort - // the stage. - // 2. The task set manager has been created but no tasks has been scheduled. In this case, - // simply abort the stage. - val taskIds = taskSetTaskIds(tsm.taskSet.id) - if (taskIds.size > 0) { - taskIds.foreach { tid => - localActor ! KillTask(tid) - } - } - tsm.error("Stage %d was cancelled".format(stageId)) - } - } - - def resourceOffer(freeCores: Int): Seq[TaskDescription] = { - synchronized { - var freeCpuCores = freeCores - 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)) - } - - var launchTask = false - for (manager <- sortedTaskSetQueue) { - do { - launchTask = false - manager.resourceOffer(null, null, freeCpuCores, null) match { - case Some(task) => - tasks += task - taskIdToTaskSetId(task.taskId) = manager.taskSet.id - taskSetTaskIds(manager.taskSet.id) += task.taskId - freeCpuCores -= 1 - launchTask = true - case None => {} - } - } while(launchTask) - } - return tasks - } - } - - def taskSetFinished(manager: TaskSetManager) { - synchronized { - activeTaskSets -= manager.taskSet.id - manager.parent.removeSchedulable(manager) - logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) - taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) - taskSetTaskIds -= manager.taskSet.id - } - } - - override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) { - if (TaskState.isFinished(state)) { - synchronized { - taskIdToTaskSetId.get(taskId) match { - case Some(taskSetId) => - val taskSetManager = activeTaskSets(taskSetId) - taskSetTaskIds(taskSetId) -= taskId - - state match { - case TaskState.FINISHED => - taskSetManager.taskEnded(taskId, state, serializedData) - case TaskState.FAILED => - taskSetManager.taskFailed(taskId, state, serializedData) - case TaskState.KILLED => - taskSetManager.error("Task %d was killed".format(taskId)) - case _ => {} - } - case None => - logInfo("Ignoring update from TID " + taskId + " because its task set is gone") - } - } - localActor ! LocalStatusUpdate(taskId, state, serializedData) - } - } - - override def stop() { - } - - override def defaultParallelism() = threads -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala deleted file mode 100644 index 53bf78267e..0000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala +++ /dev/null @@ -1,191 +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.spark.scheduler.local - -import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap - -import org.apache.spark.{ExceptionFailure, Logging, SparkEnv, SparkException, Success, TaskState} -import org.apache.spark.TaskState.TaskState -import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Pool, Schedulable, Task, - TaskDescription, TaskInfo, TaskLocality, TaskResult, TaskSet, TaskSetManager} - - -private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) - extends TaskSetManager with Logging { - - var parent: Pool = null - var weight: Int = 1 - var minShare: Int = 0 - var runningTasks: Int = 0 - var priority: Int = taskSet.priority - var stageId: Int = taskSet.stageId - var name: String = "TaskSet_" + taskSet.stageId.toString - - var failCount = new Array[Int](taskSet.tasks.size) - val taskInfos = new HashMap[Long, TaskInfo] - val numTasks = taskSet.tasks.size - var numFinished = 0 - val 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) - val MAX_TASK_FAILURES = sched.maxFailures - - def increaseRunningTasks(taskNum: Int): Unit = { - runningTasks += taskNum - if (parent != null) { - parent.increaseRunningTasks(taskNum) - } - } - - def decreaseRunningTasks(taskNum: Int): Unit = { - runningTasks -= taskNum - if (parent != null) { - parent.decreaseRunningTasks(taskNum) - } - } - - override def addSchedulable(schedulable: Schedulable): Unit = { - // nothing - } - - override def removeSchedulable(schedulable: Schedulable): Unit = { - // nothing - } - - override def getSchedulableByName(name: String): Schedulable = { - return null - } - - override def executorLost(executorId: String, host: String): Unit = { - // nothing - } - - override def checkSpeculatableTasks() = true - - override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { - var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] - sortedTaskSetQueue += this - return sortedTaskSetQueue - } - - override def hasPendingTasks() = true - - def findTask(): Option[Int] = { - for (i <- 0 to numTasks-1) { - if (copiesRunning(i) == 0 && !finished(i)) { - return Some(i) - } - } - return None - } - - override def resourceOffer( - execId: String, - host: String, - availableCpus: Int, - maxLocality: TaskLocality.TaskLocality) - : Option[TaskDescription] = - { - SparkEnv.set(sched.env) - logDebug("availableCpus:%d, numFinished:%d, numTasks:%d".format( - availableCpus.toInt, numFinished, numTasks)) - if (availableCpus > 0 && numFinished < numTasks) { - findTask() match { - case Some(index) => - val taskId = sched.attemptId.getAndIncrement() - val task = taskSet.tasks(index) - val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", - TaskLocality.NODE_LOCAL) - taskInfos(taskId) = info - // 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") - val taskName = "task %s:%d".format(taskSet.id, index) - copiesRunning(index) += 1 - increaseRunningTasks(1) - taskStarted(task, info) - return Some(new TaskDescription(taskId, null, taskName, index, bytes)) - case None => {} - } - } - return None - } - - def taskStarted(task: Task[_], info: TaskInfo) { - sched.dagScheduler.taskStarted(task, info) - } - - def taskEnded(tid: Long, state: TaskState, serializedData: ByteBuffer) { - val info = taskInfos(tid) - val index = info.index - val task = taskSet.tasks(index) - info.markSuccessful() - val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) match { - case directResult: DirectTaskResult[_] => directResult - case IndirectTaskResult(blockId) => { - throw new SparkException("Expect only DirectTaskResults when using LocalScheduler") - } - } - result.metrics.resultSize = serializedData.limit() - sched.dagScheduler.taskEnded(task, Success, result.value, result.accumUpdates, info, - result.metrics) - numFinished += 1 - decreaseRunningTasks(1) - finished(index) = true - if (numFinished == numTasks) { - sched.taskSetFinished(this) - } - } - - def taskFailed(tid: Long, state: TaskState, serializedData: ByteBuffer) { - val info = taskInfos(tid) - val index = info.index - val task = taskSet.tasks(index) - info.markFailed() - decreaseRunningTasks(1) - val reason: ExceptionFailure = ser.deserialize[ExceptionFailure]( - serializedData, getClass.getClassLoader) - sched.dagScheduler.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"))) - if (numFailures(index) > MAX_TASK_FAILURES) { - val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format( - taskSet.id, index, MAX_TASK_FAILURES, reason.description) - decreaseRunningTasks(runningTasks) - sched.dagScheduler.taskSetFailed(taskSet, errorMessage) - // need to delete failed Taskset from schedule queue - sched.taskSetFinished(this) - } - } - } - - override def error(message: String) { - sched.dagScheduler.taskSetFailed(taskSet, message) - sched.taskSetFinished(this) - } -} diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index af448fcb37..2f7d6dff38 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark -import org.scalatest.FunSuite +import org.scalatest.{BeforeAndAfterAll, FunSuite} import SparkContext._ import org.apache.spark.util.NonSerializable @@ -37,12 +37,20 @@ object FailureSuiteState { } } -class FailureSuite extends FunSuite with LocalSparkContext { +class FailureSuite extends FunSuite with LocalSparkContext with BeforeAndAfterAll { + + override def beforeAll { + System.setProperty("spark.task.maxFailures", "1") + } + + override def afterAll { + System.clearProperty("spark.task.maxFailures") + } // 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") { - sc = new SparkContext("local[1,1]", "test") + sc = new SparkContext("local[1]", "test") val results = sc.makeRDD(1 to 3, 3).map { x => FailureSuiteState.synchronized { FailureSuiteState.tasksRun += 1 @@ -62,7 +70,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { // Run a map-reduce job in which a reduce task deterministically fails once. test("failure in a two-stage job") { - sc = new SparkContext("local[1,1]", "test") + sc = new SparkContext("local[1]", "test") val results = sc.makeRDD(1 to 3).map(x => (x, x)).groupByKey(3).map { case (k, v) => FailureSuiteState.synchronized { @@ -82,7 +90,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { } test("failure because task results are not serializable") { - sc = new SparkContext("local[1,1]", "test") + sc = new SparkContext("local[1]", "test") val results = sc.makeRDD(1 to 3).map(x => new NonSerializable) val thrown = intercept[SparkException] { @@ -95,7 +103,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { } test("failure because task closure is not serializable") { - sc = new SparkContext("local[1,1]", "test") + sc = new SparkContext("local[1]", "test") val a = new NonSerializable // Non-serializable closure in the final result stage diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index f7f599532a..d9a1d6d087 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -19,23 +19,26 @@ package org.apache.spark.scheduler import scala.collection.mutable.{Buffer, HashSet} -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.scalatest.matchers.ShouldMatchers import org.apache.spark.{LocalSparkContext, SparkContext} import org.apache.spark.SparkContext._ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers - with BeforeAndAfterAll { + with BeforeAndAfter with BeforeAndAfterAll { /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 + before { + sc = new SparkContext("local", "SparkListenerSuite") + } + override def afterAll { System.clearProperty("spark.akka.frameSize") } test("basic creation of StageInfo") { - sc = new SparkContext("local", "DAGSchedulerSuite") val listener = new SaveStageInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) @@ -56,7 +59,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } test("StageInfo with fewer tasks than partitions") { - sc = new SparkContext("local", "DAGSchedulerSuite") val listener = new SaveStageInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) @@ -72,7 +74,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } test("local metrics") { - sc = new SparkContext("local", "DAGSchedulerSuite") val listener = new SaveStageInfo sc.addSparkListener(listener) sc.addSparkListener(new StatsReportListener) @@ -135,10 +136,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } test("onTaskGettingResult() called when result fetched remotely") { - // Need to use local cluster mode here, because results are not ever returned through the - // block manager when using the LocalScheduler. - sc = new SparkContext("local-cluster[1,1,512]", "test") - val listener = new SaveTaskEvents sc.addSparkListener(listener) @@ -157,10 +154,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } test("onTaskGettingResult() not called when result sent directly") { - // Need to use local cluster mode here, because results are not ever returned through the - // block manager when using the LocalScheduler. - sc = new SparkContext("local-cluster[1,1,512]", "test") - val listener = new SaveTaskEvents sc.addSparkListener(listener) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala index ee150a3107..77d3038614 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala @@ -66,9 +66,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA } before { - // Use local-cluster mode because results are returned differently when running with the - // LocalScheduler. - sc = new SparkContext("local-cluster[1,1,512]", "test") + sc = new SparkContext("local", "test") } override def afterAll { diff --git a/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala deleted file mode 100644 index 1e676c1719..0000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala +++ /dev/null @@ -1,227 +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.spark.scheduler.local - -import java.util.concurrent.Semaphore -import java.util.concurrent.CountDownLatch - -import scala.collection.mutable.HashMap - -import org.scalatest.{BeforeAndAfterEach, FunSuite} - -import org.apache.spark._ - - -class Lock() { - var finished = false - def jobWait() = { - synchronized { - while(!finished) { - this.wait() - } - } - } - - def jobFinished() = { - synchronized { - finished = true - this.notifyAll() - } - } -} - -object TaskThreadInfo { - val threadToLock = HashMap[Int, Lock]() - val threadToRunning = HashMap[Int, Boolean]() - val threadToStarted = HashMap[Int, CountDownLatch]() -} - -/* - * 1. each thread contains one job. - * 2. each job contains one stage. - * 3. each stage only contains one task. - * 4. each task(launched) must be lanched orderly(using threadToStarted) to make sure - * it will get cpu core resource, and will wait to finished after user manually - * release "Lock" and then cluster will contain another free cpu cores. - * 5. each task(pending) must use "sleep" to make sure it has been added to taskSetManager queue, - * thus it will be scheduled later when cluster has free cpu cores. - */ -class LocalSchedulerSuite extends FunSuite with LocalSparkContext with BeforeAndAfterEach { - - override def afterEach() { - super.afterEach() - System.clearProperty("spark.scheduler.mode") - } - - 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.setLocalProperty("spark.scheduler.pool", poolName) - } - override def run() { - val ans = nums.map(number => { - TaskThreadInfo.threadToRunning(number) = true - TaskThreadInfo.threadToStarted(number).countDown() - TaskThreadInfo.threadToLock(number).jobWait() - TaskThreadInfo.threadToRunning(number) = false - number - }).collect() - assert(ans.toList === List(threadIndex)) - sem.release() - } - }.start() - } - - test("Local FIFO scheduler end-to-end test") { - System.setProperty("spark.scheduler.mode", "FIFO") - sc = new SparkContext("local[4]", "test") - val sem = new Semaphore(0) - - createThread(1,null,sc,sem) - TaskThreadInfo.threadToStarted(1).await() - createThread(2,null,sc,sem) - TaskThreadInfo.threadToStarted(2).await() - createThread(3,null,sc,sem) - TaskThreadInfo.threadToStarted(3).await() - createThread(4,null,sc,sem) - TaskThreadInfo.threadToStarted(4).await() - // thread 5 and 6 (stage pending)must meet following two points - // 1. stages (taskSetManager) of jobs in thread 5 and 6 should be add to taskSetManager - // queue before executing TaskThreadInfo.threadToLock(1).jobFinished() - // 2. priority of stage in thread 5 should be prior to priority of stage in thread 6 - // So I just use "sleep" 1s here for each thread. - // TODO: any better solution? - createThread(5,null,sc,sem) - Thread.sleep(1000) - createThread(6,null,sc,sem) - Thread.sleep(1000) - - assert(TaskThreadInfo.threadToRunning(1) === true) - assert(TaskThreadInfo.threadToRunning(2) === true) - assert(TaskThreadInfo.threadToRunning(3) === true) - 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() - - assert(TaskThreadInfo.threadToRunning(1) === false) - assert(TaskThreadInfo.threadToRunning(2) === true) - assert(TaskThreadInfo.threadToRunning(3) === false) - assert(TaskThreadInfo.threadToRunning(4) === true) - assert(TaskThreadInfo.threadToRunning(5) === true) - assert(TaskThreadInfo.threadToRunning(6) === true) - - TaskThreadInfo.threadToLock(2).jobFinished() - TaskThreadInfo.threadToLock(4).jobFinished() - TaskThreadInfo.threadToLock(5).jobFinished() - TaskThreadInfo.threadToLock(6).jobFinished() - sem.acquire(6) - } - - test("Local fair scheduler end-to-end test") { - System.setProperty("spark.scheduler.mode", "FAIR") - val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) - - sc = new SparkContext("local[8]", "LocalSchedulerSuite") - val sem = new Semaphore(0) - - createThread(10,"1",sc,sem) - TaskThreadInfo.threadToStarted(10).await() - createThread(20,"2",sc,sem) - TaskThreadInfo.threadToStarted(20).await() - createThread(30,"3",sc,sem) - TaskThreadInfo.threadToStarted(30).await() - - assert(TaskThreadInfo.threadToRunning(10) === true) - assert(TaskThreadInfo.threadToRunning(20) === true) - assert(TaskThreadInfo.threadToRunning(30) === true) - - createThread(11,"1",sc,sem) - TaskThreadInfo.threadToStarted(11).await() - createThread(21,"2",sc,sem) - TaskThreadInfo.threadToStarted(21).await() - createThread(31,"3",sc,sem) - TaskThreadInfo.threadToStarted(31).await() - - assert(TaskThreadInfo.threadToRunning(11) === true) - assert(TaskThreadInfo.threadToRunning(21) === true) - assert(TaskThreadInfo.threadToRunning(31) === true) - - createThread(12,"1",sc,sem) - TaskThreadInfo.threadToStarted(12).await() - createThread(22,"2",sc,sem) - TaskThreadInfo.threadToStarted(22).await() - createThread(32,"3",sc,sem) - - assert(TaskThreadInfo.threadToRunning(12) === true) - 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 - // 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() - - assert(TaskThreadInfo.threadToRunning(23) === true) - assert(TaskThreadInfo.threadToRunning(33) === false) - - TaskThreadInfo.threadToLock(12).jobFinished() - TaskThreadInfo.threadToStarted(33).await() - - assert(TaskThreadInfo.threadToRunning(33) === true) - - TaskThreadInfo.threadToLock(20).jobFinished() - TaskThreadInfo.threadToLock(21).jobFinished() - TaskThreadInfo.threadToLock(22).jobFinished() - TaskThreadInfo.threadToLock(23).jobFinished() - TaskThreadInfo.threadToLock(30).jobFinished() - TaskThreadInfo.threadToLock(31).jobFinished() - TaskThreadInfo.threadToLock(32).jobFinished() - TaskThreadInfo.threadToLock(33).jobFinished() - - sem.acquire(11) - } -} diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala index 29b3f22e13..e873400680 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -17,16 +17,20 @@ package org.apache.spark.scheduler.cluster +import org.apache.hadoop.conf.Configuration + import org.apache.spark._ import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler} +import org.apache.spark.scheduler.TaskScheduler import org.apache.spark.util.Utils -import org.apache.hadoop.conf.Configuration /** * - * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of ApplicationMaster, etc is done + * This is a simple extension to TaskScheduler - to ensure that appropriate initialization of + * ApplicationMaster, etc. is done */ -private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) { +private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) + extends TaskScheduler(sc) { logInfo("Created YarnClusterScheduler") -- cgit v1.2.3 From a124658e53a5abeda00a2582385a294c8e452d21 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Wed, 30 Oct 2013 19:29:38 -0700 Subject: Fixed most issues with unit tests --- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 94 +++--- .../org/apache/spark/scheduler/FakeTask.scala | 26 ++ .../spark/scheduler/TaskResultGetterSuite.scala | 111 +++++++ .../spark/scheduler/TaskSchedulerSuite.scala | 265 +++++++++++++++++ .../spark/scheduler/TaskSetManagerSuite.scala | 317 ++++++++++++++++++++ .../scheduler/cluster/ClusterSchedulerSuite.scala | 267 ----------------- .../cluster/ClusterTaskSetManagerSuite.scala | 318 --------------------- .../apache/spark/scheduler/cluster/FakeTask.scala | 27 -- .../scheduler/cluster/TaskResultGetterSuite.scala | 112 -------- 9 files changed, 767 insertions(+), 770 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala delete mode 100644 core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala (limited to 'core/src') 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 00f2fdd657..394a1bb06f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -33,6 +33,24 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} +/** + * TaskScheduler that records the task sets that the DAGScheduler requested executed. + */ +class TaskSetRecordingTaskScheduler(sc: SparkContext) extends TaskScheduler(sc) { + /** Set of TaskSets the DAGScheduler has requested executed. */ + val taskSets = scala.collection.mutable.Buffer[TaskSet]() + override def start() = {} + override def stop() = {} + override def submitTasks(taskSet: TaskSet) = { + // normally done by TaskSetManager + taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) + taskSets += taskSet + } + override def cancelTasks(stageId: Int) {} + override def setDAGScheduler(dagScheduler: DAGScheduler) = {} + override def defaultParallelism() = 2 +} + /** * 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 @@ -46,24 +64,7 @@ import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} * and capturing the resulting TaskSets from the mock TaskScheduler. */ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { - - /** 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) = { - // normally done by TaskSetManager - taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) - taskSets += taskSet - } - override def cancelTasks(stageId: Int) {} - override def setDAGScheduler(dagScheduler: DAGScheduler) = {} - override def defaultParallelism() = 2 - } - + var taskScheduler: TaskSetRecordingTaskScheduler = null var mapOutputTracker: MapOutputTrackerMaster = null var scheduler: DAGScheduler = null @@ -96,7 +97,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont before { sc = new SparkContext("local", "DAGSchedulerSuite") - taskSets.clear() + taskScheduler = new TaskSetRecordingTaskScheduler(sc) + taskScheduler.taskSets.clear() cacheLocations.clear() results.clear() mapOutputTracker = new MapOutputTrackerMaster() @@ -204,7 +206,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont test("run trivial job") { val rdd = makeRdd(1, Nil) submit(rdd, Array(0)) - complete(taskSets(0), List((Success, 42))) + complete(taskScheduler.taskSets(0), List((Success, 42))) assert(results === Map(0 -> 42)) } @@ -225,7 +227,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val baseRdd = makeRdd(1, Nil) val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) submit(finalRdd, Array(0)) - complete(taskSets(0), Seq((Success, 42))) + complete(taskScheduler.taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) } @@ -235,7 +237,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont cacheLocations(baseRdd.id -> 0) = Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")) submit(finalRdd, Array(0)) - val taskSet = taskSets(0) + val taskSet = taskScheduler.taskSets(0) assertLocations(taskSet, Seq(Seq("hostA", "hostB"))) complete(taskSet, Seq((Success, 42))) assert(results === Map(0 -> 42)) @@ -243,7 +245,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont test("trivial job failure") { submit(makeRdd(1, Nil), Array(0)) - failed(taskSets(0), "some failure") + failed(taskScheduler.taskSets(0), "some failure") assert(failure.getMessage === "Job aborted: some failure") } @@ -253,12 +255,12 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(1, List(shuffleDep)) submit(reduceRdd, Array(0)) - complete(taskSets(0), Seq( + complete(taskScheduler.taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) - complete(taskSets(1), Seq((Success, 42))) + complete(taskScheduler.taskSets(1), Seq((Success, 42))) assert(results === Map(0 -> 42)) } @@ -268,11 +270,11 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) - complete(taskSets(0), Seq( + complete(taskScheduler.taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) // the 2nd ResultTask failed - complete(taskSets(1), Seq( + complete(taskScheduler.taskSets(1), Seq( (Success, 42), (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), null))) // this will get called @@ -280,10 +282,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont // ask the scheduler to try it again scheduler.resubmitFailedStages() // have the 2nd attempt pass - complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) + complete(taskScheduler.taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) // we can see both result blocks now assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === Array("hostA", "hostB")) - complete(taskSets(3), Seq((Success, 43))) + complete(taskScheduler.taskSets(3), Seq((Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) } @@ -299,7 +301,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val newEpoch = mapOutputTracker.getEpoch assert(newEpoch > oldEpoch) val noAccum = Map[Long, Any]() - val taskSet = taskSets(0) + val taskSet = taskScheduler.taskSets(0) // 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 non-failed host @@ -311,7 +313,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum, null, null)) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) - complete(taskSets(1), Seq((Success, 42), (Success, 43))) + complete(taskScheduler.taskSets(1), Seq((Success, 42), (Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) } @@ -326,14 +328,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runEvent(ExecutorLost("exec-hostA")) // DAGScheduler will immediately resubmit the stage after it appears to have no pending tasks // rather than marking it is as failed and waiting. - complete(taskSets(0), Seq( + complete(taskScheduler.taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) // have hostC complete the resubmitted task - complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) + complete(taskScheduler.taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) - complete(taskSets(2), Seq((Success, 42))) + complete(taskScheduler.taskSets(2), Seq((Success, 42))) assert(results === Map(0 -> 42)) } @@ -345,23 +347,23 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val finalRdd = makeRdd(1, List(shuffleDepTwo)) submit(finalRdd, Array(0)) // have the first stage complete normally - complete(taskSets(0), Seq( + complete(taskScheduler.taskSets(0), Seq( (Success, makeMapStatus("hostA", 2)), (Success, makeMapStatus("hostB", 2)))) // have the second stage complete normally - complete(taskSets(1), Seq( + complete(taskScheduler.taskSets(1), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostC", 1)))) // fail the third stage because hostA went down - complete(taskSets(2), Seq( + complete(taskScheduler.taskSets(2), Seq( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // have DAGScheduler try again scheduler.resubmitFailedStages() - complete(taskSets(3), Seq((Success, makeMapStatus("hostA", 2)))) - complete(taskSets(4), Seq((Success, makeMapStatus("hostA", 1)))) - complete(taskSets(5), Seq((Success, 42))) + complete(taskScheduler.taskSets(3), Seq((Success, makeMapStatus("hostA", 2)))) + complete(taskScheduler.taskSets(4), Seq((Success, makeMapStatus("hostA", 1)))) + complete(taskScheduler.taskSets(5), Seq((Success, 42))) assert(results === Map(0 -> 42)) } @@ -375,24 +377,24 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) // complete stage 2 - complete(taskSets(0), Seq( + complete(taskScheduler.taskSets(0), Seq( (Success, makeMapStatus("hostA", 2)), (Success, makeMapStatus("hostB", 2)))) // complete stage 1 - complete(taskSets(1), Seq( + complete(taskScheduler.taskSets(1), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) // pretend stage 0 failed because hostA went down - complete(taskSets(2), Seq( + complete(taskScheduler.taskSets(2), Seq( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun. scheduler.resubmitFailedStages() - assertLocations(taskSets(3), Seq(Seq("hostD"))) + assertLocations(taskScheduler.taskSets(3), Seq(Seq("hostD"))) // allow hostD to recover - complete(taskSets(3), Seq((Success, makeMapStatus("hostD", 1)))) - complete(taskSets(4), Seq((Success, 42))) + complete(taskScheduler.taskSets(3), Seq((Success, makeMapStatus("hostD", 1)))) + complete(taskScheduler.taskSets(4), Seq((Success, 42))) assert(results === Map(0 -> 42)) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala new file mode 100644 index 0000000000..0b90c4e74c --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/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 org.apache.spark.scheduler + +import org.apache.spark.TaskContext + +class FakeTask(stageId: Int, prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, 0) { + override def runTask(context: TaskContext): Int = 0 + + override def preferredLocations: Seq[TaskLocation] = prefLocs +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala new file mode 100644 index 0000000000..30e6bc5721 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 java.nio.ByteBuffer + +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} + +import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv} +import org.apache.spark.storage.TaskResultBlockId + +/** + * Removes the TaskResult from the BlockManager before delegating to a normal TaskResultGetter. + * + * Used to test the case where a BlockManager evicts the task result (or dies) before the + * TaskResult is retrieved. + */ +class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskScheduler) + extends TaskResultGetter(sparkEnv, scheduler) { + var removedResult = false + + override def enqueueSuccessfulTask( + taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) { + if (!removedResult) { + // Only remove the result once, since we'd like to test the case where the task eventually + // succeeds. + serializer.get().deserialize[TaskResult[_]](serializedData) match { + case IndirectTaskResult(blockId) => + sparkEnv.blockManager.master.removeBlock(blockId) + case directResult: DirectTaskResult[_] => + taskSetManager.abort("Internal error: expect only indirect results") + } + serializedData.rewind() + removedResult = true + } + super.enqueueSuccessfulTask(taskSetManager, tid, serializedData) + } +} + +/** + * Tests related to handling task results (both direct and indirect). + */ +class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll + with LocalSparkContext { + + override def beforeAll { + // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small + // as we can make it) so the tests don't take too long. + System.setProperty("spark.akka.frameSize", "1") + } + + before { + sc = new SparkContext("local", "test") + } + + override def afterAll { + System.clearProperty("spark.akka.frameSize") + } + + test("handling results smaller than Akka frame size") { + val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x) + assert(result === 2) + } + + test("handling results larger than Akka frame size") { + val akkaFrameSize = + sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt + val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) + assert(result === 1.to(akkaFrameSize).toArray) + + val RESULT_BLOCK_ID = TaskResultBlockId(0) + assert(sc.env.blockManager.master.getLocations(RESULT_BLOCK_ID).size === 0, + "Expect result to be removed from the block manager.") + } + + test("task retried if result missing from block manager") { + // If this test hangs, it's probably because no resource offers were made after the task + // failed. + val scheduler: TaskScheduler = sc.taskScheduler match { + case clusterScheduler: TaskScheduler => + clusterScheduler + case _ => + assert(false, "Expect local cluster to use TaskScheduler") + throw new ClassCastException + } + scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) + val akkaFrameSize = + sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt + val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) + assert(result === 1.to(akkaFrameSize).toArray) + + // Make sure two tasks were run (one failed one, and a second retried one). + assert(scheduler.nextTaskId.get() === 2) + } +} + diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerSuite.scala new file mode 100644 index 0000000000..bfbffdf261 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerSuite.scala @@ -0,0 +1,265 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 org.scalatest.FunSuite +import org.scalatest.BeforeAndAfter + +import org.apache.spark._ +import scala.collection.mutable.ArrayBuffer + +import java.util.Properties + +class FakeTaskSetManager( + initPriority: Int, + initStageId: Int, + initNumTasks: Int, + taskScheduler: TaskScheduler, + taskSet: TaskSet) + extends TaskSetManager(taskScheduler, taskSet) { + + parent = null + weight = 1 + minShare = 2 + runningTasks = 0 + priority = initPriority + stageId = initStageId + name = "TaskSet_"+stageId + override val numTasks = initNumTasks + tasksSuccessful = 0 + + def increaseRunningTasks(taskNum: Int) { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } + } + + def decreaseRunningTasks(taskNum: Int) { + runningTasks -= taskNum + if (parent != null) { + parent.decreaseRunningTasks(taskNum) + } + } + + override def addSchedulable(schedulable: Schedulable) { + } + + override def removeSchedulable(schedulable: Schedulable) { + } + + override def getSchedulableByName(name: String): Schedulable = { + return null + } + + override def executorLost(executorId: String, host: String): Unit = { + } + + override def resourceOffer( + execId: String, + host: String, + availableCpus: Int, + maxLocality: TaskLocality.TaskLocality) + : Option[TaskDescription] = + { + if (tasksSuccessful + runningTasks < numTasks) { + increaseRunningTasks(1) + return Some(new TaskDescription(0, execId, "task 0:0", 0, null)) + } + return None + } + + override def checkSpeculatableTasks(): Boolean = { + return true + } + + def taskFinished() { + decreaseRunningTasks(1) + tasksSuccessful +=1 + if (tasksSuccessful == numTasks) { + parent.removeSchedulable(this) + } + } + + def abort() { + decreaseRunningTasks(runningTasks) + parent.removeSchedulable(this) + } +} + +class TaskSchedulerSuite extends FunSuite with LocalSparkContext with Logging { + + def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: TaskScheduler, taskSet: TaskSet): FakeTaskSetManager = { + new FakeTaskSetManager(priority, stage, numTasks, cs , taskSet) + } + + def resourceOffer(rootPool: Pool): Int = { + val taskSetQueue = rootPool.getSortedTaskSetQueue() + /* Just for Test*/ + for (manager <- taskSetQueue) { + logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format( + manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks)) + } + for (taskSet <- taskSetQueue) { + taskSet.resourceOffer("execId_1", "hostname_1", 1, TaskLocality.ANY) match { + case Some(task) => + return taskSet.stageId + case None => {} + } + } + -1 + } + + def checkTaskSetId(rootPool: Pool, expectedTaskSetId: Int) { + assert(resourceOffer(rootPool) === expectedTaskSetId) + } + + test("FIFO Scheduler Test") { + sc = new SparkContext("local", "TaskSchedulerSuite") + val taskScheduler = new TaskScheduler(sc) + var tasks = ArrayBuffer[Task[_]]() + val task = new FakeTask(0) + tasks += task + val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + + val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0) + val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) + schedulableBuilder.buildPools() + + val taskSetManager0 = createDummyTaskSetManager(0, 0, 2, taskScheduler, taskSet) + val taskSetManager1 = createDummyTaskSetManager(0, 1, 2, taskScheduler, taskSet) + val taskSetManager2 = createDummyTaskSetManager(0, 2, 2, taskScheduler, taskSet) + schedulableBuilder.addTaskSetManager(taskSetManager0, null) + schedulableBuilder.addTaskSetManager(taskSetManager1, null) + schedulableBuilder.addTaskSetManager(taskSetManager2, null) + + checkTaskSetId(rootPool, 0) + resourceOffer(rootPool) + checkTaskSetId(rootPool, 1) + resourceOffer(rootPool) + taskSetManager1.abort() + checkTaskSetId(rootPool, 2) + } + + test("Fair Scheduler Test") { + sc = new SparkContext("local", "TaskSchedulerSuite") + val taskScheduler = new TaskScheduler(sc) + var tasks = ArrayBuffer[Task[_]]() + val task = new FakeTask(0) + tasks += task + val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + + val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() + System.setProperty("spark.scheduler.allocation.file", xmlPath) + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val schedulableBuilder = new FairSchedulableBuilder(rootPool) + schedulableBuilder.buildPools() + + assert(rootPool.getSchedulableByName("default") != null) + assert(rootPool.getSchedulableByName("1") != null) + assert(rootPool.getSchedulableByName("2") != null) + assert(rootPool.getSchedulableByName("3") != null) + assert(rootPool.getSchedulableByName("1").minShare === 2) + assert(rootPool.getSchedulableByName("1").weight === 1) + assert(rootPool.getSchedulableByName("2").minShare === 3) + assert(rootPool.getSchedulableByName("2").weight === 1) + assert(rootPool.getSchedulableByName("3").minShare === 0) + assert(rootPool.getSchedulableByName("3").weight === 1) + + val properties1 = new Properties() + properties1.setProperty("spark.scheduler.pool","1") + val properties2 = new Properties() + properties2.setProperty("spark.scheduler.pool","2") + + val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, taskScheduler, taskSet) + val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, taskScheduler, taskSet) + val taskSetManager12 = createDummyTaskSetManager(1, 2, 2, taskScheduler, taskSet) + schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) + schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) + schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) + + val taskSetManager23 = createDummyTaskSetManager(2, 3, 2, taskScheduler, taskSet) + val taskSetManager24 = createDummyTaskSetManager(2, 4, 2, taskScheduler, taskSet) + schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) + schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) + + checkTaskSetId(rootPool, 0) + checkTaskSetId(rootPool, 3) + checkTaskSetId(rootPool, 3) + checkTaskSetId(rootPool, 1) + checkTaskSetId(rootPool, 4) + checkTaskSetId(rootPool, 2) + checkTaskSetId(rootPool, 2) + checkTaskSetId(rootPool, 4) + + taskSetManager12.taskFinished() + assert(rootPool.getSchedulableByName("1").runningTasks === 3) + taskSetManager24.abort() + assert(rootPool.getSchedulableByName("2").runningTasks === 2) + } + + test("Nested Pool Test") { + sc = new SparkContext("local", "TaskSchedulerSuite") + val taskScheduler = new TaskScheduler(sc) + var tasks = ArrayBuffer[Task[_]]() + val task = new FakeTask(0) + tasks += task + val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1) + val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1) + rootPool.addSchedulable(pool0) + rootPool.addSchedulable(pool1) + + val pool00 = new Pool("00", SchedulingMode.FAIR, 2, 2) + val pool01 = new Pool("01", SchedulingMode.FAIR, 1, 1) + pool0.addSchedulable(pool00) + pool0.addSchedulable(pool01) + + val pool10 = new Pool("10", SchedulingMode.FAIR, 2, 2) + val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1) + pool1.addSchedulable(pool10) + pool1.addSchedulable(pool11) + + val taskSetManager000 = createDummyTaskSetManager(0, 0, 5, taskScheduler, taskSet) + val taskSetManager001 = createDummyTaskSetManager(0, 1, 5, taskScheduler, taskSet) + pool00.addSchedulable(taskSetManager000) + pool00.addSchedulable(taskSetManager001) + + val taskSetManager010 = createDummyTaskSetManager(1, 2, 5, taskScheduler, taskSet) + val taskSetManager011 = createDummyTaskSetManager(1, 3, 5, taskScheduler, taskSet) + pool01.addSchedulable(taskSetManager010) + pool01.addSchedulable(taskSetManager011) + + val taskSetManager100 = createDummyTaskSetManager(2, 4, 5, taskScheduler, taskSet) + val taskSetManager101 = createDummyTaskSetManager(2, 5, 5, taskScheduler, taskSet) + pool10.addSchedulable(taskSetManager100) + pool10.addSchedulable(taskSetManager101) + + val taskSetManager110 = createDummyTaskSetManager(3, 6, 5, taskScheduler, taskSet) + val taskSetManager111 = createDummyTaskSetManager(3, 7, 5, taskScheduler, taskSet) + pool11.addSchedulable(taskSetManager110) + pool11.addSchedulable(taskSetManager111) + + checkTaskSetId(rootPool, 0) + checkTaskSetId(rootPool, 4) + checkTaskSetId(rootPool, 6) + checkTaskSetId(rootPool, 2) + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala new file mode 100644 index 0000000000..fe3ea7b594 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 scala.collection.mutable.ArrayBuffer +import scala.collection.mutable + +import org.scalatest.FunSuite + +import org.apache.spark._ +import org.apache.spark.executor.TaskMetrics +import java.nio.ByteBuffer +import org.apache.spark.util.{Utils, FakeClock} + +class FakeDAGScheduler(taskScheduler: FakeTaskScheduler) extends DAGScheduler(taskScheduler) { + override def taskStarted(task: Task[_], taskInfo: TaskInfo) { + taskScheduler.startedTasks += taskInfo.index + } + + override def taskEnded( + task: Task[_], + reason: TaskEndReason, + result: Any, + accumUpdates: mutable.Map[Long, Any], + taskInfo: TaskInfo, + taskMetrics: TaskMetrics) { + taskScheduler.endedTasks(taskInfo.index) = reason + } + + override def executorGained(execId: String, host: String) {} + + override def executorLost(execId: String) {} + + override def taskSetFailed(taskSet: TaskSet, reason: String) { + taskScheduler.taskSetsFailed += taskSet.id + } +} + +/** + * A mock TaskScheduler 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 TaskSetManager. + */ +class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* execId, host */) + extends TaskScheduler(sc) +{ + val startedTasks = new ArrayBuffer[Long] + val endedTasks = new mutable.HashMap[Long, TaskEndReason] + val finishedManagers = new ArrayBuffer[TaskSetManager] + val taskSetsFailed = new ArrayBuffer[String] + + val executors = new mutable.HashMap[String, String] ++ liveExecutors + + dagScheduler = new FakeDAGScheduler(this) + + 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 TaskSetManagerSuite 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 FakeTaskScheduler(sc, ("exec1", "host1")) + val taskSet = createTaskSet(1) + val manager = new TaskSetManager(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.handleSuccessfulTask(0, 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 FakeTaskScheduler(sc, ("exec1", "host1")) + val taskSet = createTaskSet(3) + val manager = new TaskSetManager(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.handleSuccessfulTask(0, createTaskResult(0)) + manager.handleSuccessfulTask(1, createTaskResult(1)) + assert(sched.endedTasks(0) === Success) + assert(sched.endedTasks(1) === Success) + assert(!sched.finishedManagers.contains(manager)) + + // Finish the last task + manager.handleSuccessfulTask(2, createTaskResult(2)) + assert(sched.endedTasks(2) === Success) + assert(sched.finishedManagers.contains(manager)) + } + + test("basic delay scheduling") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(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 TaskSetManager(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 FakeTaskScheduler(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 TaskSetManager(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 FakeTaskScheduler(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 TaskSetManager(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) + } + + test("task result lost") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + val taskSet = createTaskSet(1) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, clock) + + assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) + + // Tell it the task has finished but the result was lost. + manager.handleFailedTask(0, TaskState.FINISHED, Some(TaskResultLost)) + assert(sched.endedTasks(0) === TaskResultLost) + + // Re-offer the host -- now we should get task 0 again. + assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) + } + + test("repeated failures lead to task set abortion") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + val taskSet = createTaskSet(1) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, clock) + + // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted + // after the last failure. + (0 until manager.MAX_TASK_FAILURES).foreach { index => + val offerResult = manager.resourceOffer("exec1", "host1", 1, ANY) + assert(offerResult != None, + "Expect resource offer on iteration %s to return a task".format(index)) + assert(offerResult.get.index === 0) + manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, Some(TaskResultLost)) + if (index < manager.MAX_TASK_FAILURES) { + assert(!sched.taskSetsFailed.contains(taskSet.id)) + } else { + assert(sched.taskSetsFailed.contains(taskSet.id)) + } + } + } + + + /** + * 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): DirectTaskResult[Int] = { + new DirectTaskResult[Int](id, mutable.Map.empty, new TaskMetrics) + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala deleted file mode 100644 index 95d3553d91..0000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala +++ /dev/null @@ -1,267 +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.spark.scheduler.cluster - -import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter - -import org.apache.spark._ -import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster._ -import scala.collection.mutable.ArrayBuffer - -import java.util.Properties - -class FakeTaskSetManager( - initPriority: Int, - initStageId: Int, - initNumTasks: Int, - clusterScheduler: ClusterScheduler, - taskSet: TaskSet) - extends ClusterTaskSetManager(clusterScheduler, taskSet) { - - parent = null - weight = 1 - minShare = 2 - runningTasks = 0 - priority = initPriority - stageId = initStageId - name = "TaskSet_"+stageId - override val numTasks = initNumTasks - tasksSuccessful = 0 - - def increaseRunningTasks(taskNum: Int) { - runningTasks += taskNum - if (parent != null) { - parent.increaseRunningTasks(taskNum) - } - } - - def decreaseRunningTasks(taskNum: Int) { - runningTasks -= taskNum - if (parent != null) { - parent.decreaseRunningTasks(taskNum) - } - } - - override def addSchedulable(schedulable: Schedulable) { - } - - override def removeSchedulable(schedulable: Schedulable) { - } - - override def getSchedulableByName(name: String): Schedulable = { - return null - } - - override def executorLost(executorId: String, host: String): Unit = { - } - - override def resourceOffer( - execId: String, - host: String, - availableCpus: Int, - maxLocality: TaskLocality.TaskLocality) - : Option[TaskDescription] = - { - if (tasksSuccessful + runningTasks < numTasks) { - increaseRunningTasks(1) - return Some(new TaskDescription(0, execId, "task 0:0", 0, null)) - } - return None - } - - override def checkSpeculatableTasks(): Boolean = { - return true - } - - def taskFinished() { - decreaseRunningTasks(1) - tasksSuccessful +=1 - if (tasksSuccessful == numTasks) { - parent.removeSchedulable(this) - } - } - - def abort() { - decreaseRunningTasks(runningTasks) - parent.removeSchedulable(this) - } -} - -class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging { - - 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 = { - val taskSetQueue = rootPool.getSortedTaskSetQueue() - /* Just for Test*/ - for (manager <- taskSetQueue) { - logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format( - manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks)) - } - for (taskSet <- taskSetQueue) { - taskSet.resourceOffer("execId_1", "hostname_1", 1, TaskLocality.ANY) match { - case Some(task) => - return taskSet.stageId - case None => {} - } - } - -1 - } - - def checkTaskSetId(rootPool: Pool, expectedTaskSetId: Int) { - assert(resourceOffer(rootPool) === expectedTaskSetId) - } - - test("FIFO Scheduler Test") { - sc = new SparkContext("local", "ClusterSchedulerSuite") - val clusterScheduler = new ClusterScheduler(sc) - var tasks = ArrayBuffer[Task[_]]() - val task = new FakeTask(0) - tasks += task - val taskSet = new TaskSet(tasks.toArray,0,0,0,null) - - val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0) - val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) - schedulableBuilder.buildPools() - - val taskSetManager0 = createDummyTaskSetManager(0, 0, 2, clusterScheduler, taskSet) - val taskSetManager1 = createDummyTaskSetManager(0, 1, 2, clusterScheduler, taskSet) - val taskSetManager2 = createDummyTaskSetManager(0, 2, 2, clusterScheduler, taskSet) - schedulableBuilder.addTaskSetManager(taskSetManager0, null) - schedulableBuilder.addTaskSetManager(taskSetManager1, null) - schedulableBuilder.addTaskSetManager(taskSetManager2, null) - - checkTaskSetId(rootPool, 0) - resourceOffer(rootPool) - checkTaskSetId(rootPool, 1) - resourceOffer(rootPool) - taskSetManager1.abort() - checkTaskSetId(rootPool, 2) - } - - test("Fair Scheduler Test") { - sc = new SparkContext("local", "ClusterSchedulerSuite") - val clusterScheduler = new ClusterScheduler(sc) - var tasks = ArrayBuffer[Task[_]]() - val task = new FakeTask(0) - tasks += task - val taskSet = new TaskSet(tasks.toArray,0,0,0,null) - - val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) - val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) - val schedulableBuilder = new FairSchedulableBuilder(rootPool) - schedulableBuilder.buildPools() - - assert(rootPool.getSchedulableByName("default") != null) - assert(rootPool.getSchedulableByName("1") != null) - assert(rootPool.getSchedulableByName("2") != null) - assert(rootPool.getSchedulableByName("3") != null) - assert(rootPool.getSchedulableByName("1").minShare === 2) - assert(rootPool.getSchedulableByName("1").weight === 1) - assert(rootPool.getSchedulableByName("2").minShare === 3) - assert(rootPool.getSchedulableByName("2").weight === 1) - assert(rootPool.getSchedulableByName("3").minShare === 0) - assert(rootPool.getSchedulableByName("3").weight === 1) - - val properties1 = new Properties() - properties1.setProperty("spark.scheduler.pool","1") - val properties2 = new Properties() - properties2.setProperty("spark.scheduler.pool","2") - - val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, clusterScheduler, taskSet) - val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, clusterScheduler, taskSet) - val taskSetManager12 = createDummyTaskSetManager(1, 2, 2, clusterScheduler, taskSet) - schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) - schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) - schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) - - val taskSetManager23 = createDummyTaskSetManager(2, 3, 2, clusterScheduler, taskSet) - val taskSetManager24 = createDummyTaskSetManager(2, 4, 2, clusterScheduler, taskSet) - schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) - schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) - - checkTaskSetId(rootPool, 0) - checkTaskSetId(rootPool, 3) - checkTaskSetId(rootPool, 3) - checkTaskSetId(rootPool, 1) - checkTaskSetId(rootPool, 4) - checkTaskSetId(rootPool, 2) - checkTaskSetId(rootPool, 2) - checkTaskSetId(rootPool, 4) - - taskSetManager12.taskFinished() - assert(rootPool.getSchedulableByName("1").runningTasks === 3) - taskSetManager24.abort() - assert(rootPool.getSchedulableByName("2").runningTasks === 2) - } - - test("Nested Pool Test") { - sc = new SparkContext("local", "ClusterSchedulerSuite") - val clusterScheduler = new ClusterScheduler(sc) - var tasks = ArrayBuffer[Task[_]]() - val task = new FakeTask(0) - tasks += task - val taskSet = new TaskSet(tasks.toArray,0,0,0,null) - - val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) - val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1) - val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1) - rootPool.addSchedulable(pool0) - rootPool.addSchedulable(pool1) - - val pool00 = new Pool("00", SchedulingMode.FAIR, 2, 2) - val pool01 = new Pool("01", SchedulingMode.FAIR, 1, 1) - pool0.addSchedulable(pool00) - pool0.addSchedulable(pool01) - - val pool10 = new Pool("10", SchedulingMode.FAIR, 2, 2) - val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1) - pool1.addSchedulable(pool10) - pool1.addSchedulable(pool11) - - val taskSetManager000 = createDummyTaskSetManager(0, 0, 5, clusterScheduler, taskSet) - val taskSetManager001 = createDummyTaskSetManager(0, 1, 5, clusterScheduler, taskSet) - pool00.addSchedulable(taskSetManager000) - pool00.addSchedulable(taskSetManager001) - - val taskSetManager010 = createDummyTaskSetManager(1, 2, 5, clusterScheduler, taskSet) - val taskSetManager011 = createDummyTaskSetManager(1, 3, 5, clusterScheduler, taskSet) - pool01.addSchedulable(taskSetManager010) - pool01.addSchedulable(taskSetManager011) - - val taskSetManager100 = createDummyTaskSetManager(2, 4, 5, clusterScheduler, taskSet) - val taskSetManager101 = createDummyTaskSetManager(2, 5, 5, clusterScheduler, taskSet) - pool10.addSchedulable(taskSetManager100) - pool10.addSchedulable(taskSetManager101) - - val taskSetManager110 = createDummyTaskSetManager(3, 6, 5, clusterScheduler, taskSet) - val taskSetManager111 = createDummyTaskSetManager(3, 7, 5, clusterScheduler, taskSet) - pool11.addSchedulable(taskSetManager110) - pool11.addSchedulable(taskSetManager111) - - checkTaskSetId(rootPool, 0) - checkTaskSetId(rootPool, 4) - checkTaskSetId(rootPool, 6) - checkTaskSetId(rootPool, 2) - } -} 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 deleted file mode 100644 index b97f2b19b5..0000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala +++ /dev/null @@ -1,318 +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.spark.scheduler.cluster - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable - -import org.scalatest.FunSuite - -import org.apache.spark._ -import org.apache.spark.scheduler._ -import org.apache.spark.executor.TaskMetrics -import java.nio.ByteBuffer -import org.apache.spark.util.{Utils, FakeClock} - -class FakeDAGScheduler(taskScheduler: FakeClusterScheduler) extends DAGScheduler(taskScheduler) { - override def taskStarted(task: Task[_], taskInfo: TaskInfo) { - taskScheduler.startedTasks += taskInfo.index - } - - override def taskEnded( - task: Task[_], - reason: TaskEndReason, - result: Any, - accumUpdates: mutable.Map[Long, Any], - taskInfo: TaskInfo, - taskMetrics: TaskMetrics) { - taskScheduler.endedTasks(taskInfo.index) = reason - } - - override def executorGained(execId: String, host: String) {} - - override def executorLost(execId: String) {} - - override def taskSetFailed(taskSet: TaskSet, reason: String) { - taskScheduler.taskSetsFailed += taskSet.id - } -} - -/** - * 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 taskSetsFailed = new ArrayBuffer[String] - - val executors = new mutable.HashMap[String, String] ++ liveExecutors - - dagScheduler = new FakeDAGScheduler(this) - - 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.handleSuccessfulTask(0, 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.handleSuccessfulTask(0, createTaskResult(0)) - manager.handleSuccessfulTask(1, createTaskResult(1)) - assert(sched.endedTasks(0) === Success) - assert(sched.endedTasks(1) === Success) - assert(!sched.finishedManagers.contains(manager)) - - // Finish the last task - manager.handleSuccessfulTask(2, 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) - } - - test("task result lost") { - sc = new SparkContext("local", "test") - val sched = new FakeClusterScheduler(sc, ("exec1", "host1")) - val taskSet = createTaskSet(1) - val clock = new FakeClock - val manager = new ClusterTaskSetManager(sched, taskSet, clock) - - assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) - - // Tell it the task has finished but the result was lost. - manager.handleFailedTask(0, TaskState.FINISHED, Some(TaskResultLost)) - assert(sched.endedTasks(0) === TaskResultLost) - - // Re-offer the host -- now we should get task 0 again. - assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) - } - - test("repeated failures lead to task set abortion") { - sc = new SparkContext("local", "test") - val sched = new FakeClusterScheduler(sc, ("exec1", "host1")) - val taskSet = createTaskSet(1) - val clock = new FakeClock - val manager = new ClusterTaskSetManager(sched, taskSet, clock) - - // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted - // after the last failure. - (0 until manager.MAX_TASK_FAILURES).foreach { index => - val offerResult = manager.resourceOffer("exec1", "host1", 1, ANY) - assert(offerResult != None, - "Expect resource offer on iteration %s to return a task".format(index)) - assert(offerResult.get.index === 0) - manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, Some(TaskResultLost)) - if (index < manager.MAX_TASK_FAILURES) { - assert(!sched.taskSetsFailed.contains(taskSet.id)) - } else { - assert(sched.taskSetsFailed.contains(taskSet.id)) - } - } - } - - - /** - * 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): DirectTaskResult[Int] = { - new DirectTaskResult[Int](id, mutable.Map.empty, new TaskMetrics) - } -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala deleted file mode 100644 index 0f01515179..0000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.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.spark.scheduler.cluster - -import org.apache.spark.TaskContext -import org.apache.spark.scheduler.{TaskLocation, Task} - -class FakeTask(stageId: Int, prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, 0) { - override def runTask(context: TaskContext): Int = 0 - - override def preferredLocations: Seq[TaskLocation] = prefLocs -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala deleted file mode 100644 index 77d3038614..0000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala +++ /dev/null @@ -1,112 +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.spark.scheduler.cluster - -import java.nio.ByteBuffer - -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} - -import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv} -import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, TaskResult} -import org.apache.spark.storage.TaskResultBlockId - -/** - * Removes the TaskResult from the BlockManager before delegating to a normal TaskResultGetter. - * - * Used to test the case where a BlockManager evicts the task result (or dies) before the - * TaskResult is retrieved. - */ -class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler) - extends TaskResultGetter(sparkEnv, scheduler) { - var removedResult = false - - override def enqueueSuccessfulTask( - taskSetManager: ClusterTaskSetManager, tid: Long, serializedData: ByteBuffer) { - if (!removedResult) { - // Only remove the result once, since we'd like to test the case where the task eventually - // succeeds. - serializer.get().deserialize[TaskResult[_]](serializedData) match { - case IndirectTaskResult(blockId) => - sparkEnv.blockManager.master.removeBlock(blockId) - case directResult: DirectTaskResult[_] => - taskSetManager.abort("Internal error: expect only indirect results") - } - serializedData.rewind() - removedResult = true - } - super.enqueueSuccessfulTask(taskSetManager, tid, serializedData) - } -} - -/** - * Tests related to handling task results (both direct and indirect). - */ -class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll - with LocalSparkContext { - - override def beforeAll { - // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small - // as we can make it) so the tests don't take too long. - System.setProperty("spark.akka.frameSize", "1") - } - - before { - sc = new SparkContext("local", "test") - } - - override def afterAll { - System.clearProperty("spark.akka.frameSize") - } - - test("handling results smaller than Akka frame size") { - val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x) - assert(result === 2) - } - - test("handling results larger than Akka frame size") { - val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt - val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) - assert(result === 1.to(akkaFrameSize).toArray) - - val RESULT_BLOCK_ID = TaskResultBlockId(0) - assert(sc.env.blockManager.master.getLocations(RESULT_BLOCK_ID).size === 0, - "Expect result to be removed from the block manager.") - } - - test("task retried if result missing from block manager") { - // If this test hangs, it's probably because no resource offers were made after the task - // failed. - val scheduler: ClusterScheduler = sc.taskScheduler match { - case clusterScheduler: ClusterScheduler => - clusterScheduler - case _ => - assert(false, "Expect local cluster to use ClusterScheduler") - throw new ClassCastException - } - scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) - val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt - val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) - assert(result === 1.to(akkaFrameSize).toArray) - - // Make sure two tasks were run (one failed one, and a second retried one). - assert(scheduler.nextTaskId.get() === 2) - } -} - -- cgit v1.2.3 From fb64828b0b573f3a77938592f168af7aa3a2b6c5 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Thu, 31 Oct 2013 23:42:56 -0700 Subject: Cleaned up imports and fixed test bug --- .../main/scala/org/apache/spark/scheduler/TaskScheduler.scala | 3 +-- .../main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 1 - .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 9 +++++---- 3 files changed, 6 insertions(+), 7 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 3f694dd25d..b4ec695ece 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -27,7 +27,6 @@ import scala.collection.mutable.HashSet import org.apache.spark._ import org.apache.spark.TaskState.TaskState -import org.apache.spark.scheduler._ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode /** @@ -449,7 +448,7 @@ private[spark] class TaskScheduler(val sc: SparkContext, isLocal: Boolean = fals } -object TaskScheduler { +private[spark] object TaskScheduler { /** * Used to balance containers across hosts. * diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 13271b10f3..90b6519027 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -28,7 +28,6 @@ import scala.math.min import org.apache.spark.{ExceptionFailure, FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskKilled, TaskResultLost, TaskState} import org.apache.spark.TaskState.TaskState -import org.apache.spark.scheduler._ import org.apache.spark.util.{SystemClock, Clock} 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 394a1bb06f..5b5a2178f3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -36,14 +36,15 @@ import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} /** * TaskScheduler that records the task sets that the DAGScheduler requested executed. */ -class TaskSetRecordingTaskScheduler(sc: SparkContext) extends TaskScheduler(sc) { +class TaskSetRecordingTaskScheduler(sc: SparkContext, + mapOutputTrackerMaster: MapOutputTrackerMaster) extends TaskScheduler(sc) { /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() override def start() = {} override def stop() = {} override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager - taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) + taskSet.tasks.foreach(_.epoch = mapOutputTrackerMaster.getEpoch) taskSets += taskSet } override def cancelTasks(stageId: Int) {} @@ -97,11 +98,11 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont before { sc = new SparkContext("local", "DAGSchedulerSuite") - taskScheduler = new TaskSetRecordingTaskScheduler(sc) + mapOutputTracker = new MapOutputTrackerMaster() + taskScheduler = new TaskSetRecordingTaskScheduler(sc, mapOutputTracker) taskScheduler.taskSets.clear() cacheLocations.clear() results.clear() - mapOutputTracker = new MapOutputTrackerMaster() scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) { override def runLocally(job: ActiveJob) { // don't bother with the thread while unit testing -- cgit v1.2.3 From a48d88d206fae348720ab077a624b3c57293374f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 2 Nov 2013 21:13:18 -0700 Subject: Replace magic lengths with constants in PySpark. Write the length of the accumulators section up-front rather than terminating it with a negative length. I find this easier to read. --- .../org/apache/spark/api/python/PythonRDD.scala | 26 +++++++++++++--------- python/pyspark/serializers.py | 6 +++++ python/pyspark/worker.py | 13 ++++++----- 3 files changed, 29 insertions(+), 16 deletions(-) (limited to 'core/src') 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 12b4d94a56..0d5913ec60 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 @@ -132,7 +132,7 @@ private[spark] class PythonRDD[T: ClassManifest]( val obj = new Array[Byte](length) stream.readFully(obj) obj - case -3 => + case SpecialLengths.TIMING_DATA => // Timing data from worker val bootTime = stream.readLong() val initTime = stream.readLong() @@ -143,24 +143,24 @@ private[spark] class PythonRDD[T: ClassManifest]( val total = finishTime - startTime logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, init, finish)) read - case -2 => + case SpecialLengths.PYTHON_EXCEPTION_THROWN => // Signals that an exception has been thrown in python val exLength = stream.readInt() val obj = new Array[Byte](exLength) stream.readFully(obj) throw new PythonException(new String(obj)) - case -1 => + case SpecialLengths.END_OF_DATA_SECTION => // We've finished the data section of the output, but we can still - // read some accumulator updates; let's do that, breaking when we - // get a negative length record. - var len2 = stream.readInt() - while (len2 >= 0) { - val update = new Array[Byte](len2) + // read some accumulator updates: + val numAccumulatorUpdates = stream.readInt() + (1 to numAccumulatorUpdates).foreach { _ => + val updateLen = stream.readInt() + val update = new Array[Byte](updateLen) stream.readFully(update) accumulator += Collections.singletonList(update) - len2 = stream.readInt() + } - new Array[Byte](0) + Array.empty[Byte] } } catch { case eof: EOFException => { @@ -197,6 +197,12 @@ private class PairwiseRDD(prev: RDD[Array[Byte]]) extends val asJavaPairRDD : JavaPairRDD[Long, Array[Byte]] = JavaPairRDD.fromRDD(this) } +private object SpecialLengths { + val END_OF_DATA_SECTION = -1 + val PYTHON_EXCEPTION_THROWN = -2 + val TIMING_DATA = -3 +} + private[spark] object PythonRDD { /** Strips the pickle PROTO and STOP opcodes from the start and end of a pickle */ diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 54fed1c9c7..fbc280fd37 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -19,6 +19,12 @@ import struct import cPickle +class SpecialLengths(object): + END_OF_DATA_SECTION = -1 + PYTHON_EXCEPTION_THROWN = -2 + TIMING_DATA = -3 + + class Batch(object): """ Used to store multiple RDD entries as a single Java object. diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index d63c2aaef7..7696df9d1c 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -31,7 +31,8 @@ from pyspark.broadcast import Broadcast, _broadcastRegistry from pyspark.cloudpickle import CloudPickler from pyspark.files import SparkFiles from pyspark.serializers import write_with_length, read_with_length, write_int, \ - read_long, write_long, read_int, dump_pickle, load_pickle, read_from_pickle_file + read_long, write_long, read_int, dump_pickle, load_pickle, read_from_pickle_file \ + SpecialLengths def load_obj(infile): @@ -39,7 +40,7 @@ def load_obj(infile): def report_times(outfile, boot, init, finish): - write_int(-3, outfile) + write_int(SpecialLengths.TIMING_DATA, outfile) write_long(1000 * boot, outfile) write_long(1000 * init, outfile) write_long(1000 * finish, outfile) @@ -82,16 +83,16 @@ def main(infile, outfile): for obj in func(split_index, iterator): write_with_length(dumps(obj), outfile) except Exception as e: - write_int(-2, outfile) + write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) write_with_length(traceback.format_exc(), outfile) sys.exit(-1) finish_time = time.time() report_times(outfile, boot_time, init_time, finish_time) # Mark the beginning of the accumulators section of the output - write_int(-1, outfile) - for aid, accum in _accumulatorRegistry.items(): + write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) + write_int(len(_accumulatorRegistry), outfile) + for (aid, accum) in _accumulatorRegistry.items(): write_with_length(dump_pickle((aid, accum._value)), outfile) - write_int(-1, outfile) if __name__ == '__main__': -- cgit v1.2.3 From 7d68a81a8ed5f49fefb3bd0fa0b9d3835cc7d86e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 3 Nov 2013 11:03:02 -0800 Subject: Remove Pickle-wrapping of Java objects in PySpark. If we support custom serializers, the Python worker will know what type of input to expect, so we won't need to wrap Tuple2 and Strings into pickled tuples and strings. --- .../org/apache/spark/api/python/PythonRDD.scala | 106 ++++++++------------- python/pyspark/context.py | 10 +- python/pyspark/rdd.py | 11 ++- python/pyspark/serializers.py | 18 ++++ python/pyspark/worker.py | 14 ++- 5 files changed, 78 insertions(+), 81 deletions(-) (limited to 'core/src') 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 0d5913ec60..eb0b0db0cc 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 @@ -75,7 +75,7 @@ private[spark] class PythonRDD[T: ClassManifest]( // Partition index dataOut.writeInt(split.index) // sparkFilesDir - PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dataOut) + dataOut.writeUTF(SparkFiles.getRootDirectory) // Broadcast variables dataOut.writeInt(broadcastVars.length) for (broadcast <- broadcastVars) { @@ -85,9 +85,7 @@ private[spark] class PythonRDD[T: ClassManifest]( } // Python includes (*.zip and *.egg files) dataOut.writeInt(pythonIncludes.length) - for (f <- pythonIncludes) { - PythonRDD.writeAsPickle(f, dataOut) - } + pythonIncludes.foreach(dataOut.writeUTF) dataOut.flush() // Serialized user code for (elem <- command) { @@ -96,7 +94,7 @@ private[spark] class PythonRDD[T: ClassManifest]( printOut.flush() // Data values for (elem <- parent.iterator(split, context)) { - PythonRDD.writeAsPickle(elem, dataOut) + PythonRDD.writeToStream(elem, dataOut) } dataOut.flush() printOut.flush() @@ -205,60 +203,7 @@ private object SpecialLengths { private[spark] object PythonRDD { - /** Strips the pickle PROTO and STOP opcodes from the start and end of a pickle */ - def stripPickle(arr: Array[Byte]) : Array[Byte] = { - arr.slice(2, arr.length - 1) - } - - /** - * Write strings, pickled Python objects, or pairs of pickled objects to a data output stream. - * The data format is a 32-bit integer representing the pickled object's length (in bytes), - * followed by the pickled data. - * - * Pickle module: - * - * http://docs.python.org/2/library/pickle.html - * - * The pickle protocol is documented in the source of the `pickle` and `pickletools` modules: - * - * http://hg.python.org/cpython/file/2.6/Lib/pickle.py - * http://hg.python.org/cpython/file/2.6/Lib/pickletools.py - * - * @param elem the object to write - * @param dOut a data output stream - */ - def writeAsPickle(elem: Any, dOut: DataOutputStream) { - if (elem.isInstanceOf[Array[Byte]]) { - val arr = elem.asInstanceOf[Array[Byte]] - dOut.writeInt(arr.length) - dOut.write(arr) - } else if (elem.isInstanceOf[scala.Tuple2[Array[Byte], Array[Byte]]]) { - 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) - dOut.writeByte(Pickle.PROTO) - dOut.writeByte(Pickle.TWO) - dOut.write(PythonRDD.stripPickle(t._1)) - dOut.write(PythonRDD.stripPickle(t._2)) - dOut.writeByte(Pickle.TUPLE2) - dOut.writeByte(Pickle.STOP) - } else if (elem.isInstanceOf[String]) { - // For uniformity, strings are wrapped into Pickles. - val s = elem.asInstanceOf[String].getBytes("UTF-8") - val length = 2 + 1 + 4 + s.length + 1 - dOut.writeInt(length) - dOut.writeByte(Pickle.PROTO) - dOut.writeByte(Pickle.TWO) - dOut.write(Pickle.BINUNICODE) - dOut.writeInt(Integer.reverseBytes(s.length)) - dOut.write(s) - dOut.writeByte(Pickle.STOP) - } else { - throw new SparkException("Unexpected RDD type") - } - } - - def readRDDFromPickleFile(sc: JavaSparkContext, filename: String, parallelism: Int) : + def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): JavaRDD[Array[Byte]] = { val file = new DataInputStream(new FileInputStream(filename)) val objs = new collection.mutable.ArrayBuffer[Array[Byte]] @@ -276,15 +221,46 @@ private[spark] object PythonRDD { JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } - def writeIteratorToPickleFile[T](items: java.util.Iterator[T], filename: String) { + def writeStringAsPickle(elem: String, dOut: DataOutputStream) { + val s = elem.getBytes("UTF-8") + val length = 2 + 1 + 4 + s.length + 1 + dOut.writeInt(length) + dOut.writeByte(Pickle.PROTO) + dOut.writeByte(Pickle.TWO) + dOut.write(Pickle.BINUNICODE) + dOut.writeInt(Integer.reverseBytes(s.length)) + dOut.write(s) + dOut.writeByte(Pickle.STOP) + } + + def writeToStream(elem: Any, dataOut: DataOutputStream) { + elem match { + case bytes: Array[Byte] => + dataOut.writeInt(bytes.length) + dataOut.write(bytes) + case pair: (Array[Byte], Array[Byte]) => + dataOut.writeInt(pair._1.length) + dataOut.write(pair._1) + dataOut.writeInt(pair._2.length) + dataOut.write(pair._2) + case str: String => + // Until we've implemented full custom serializer support, we need to return + // strings as Pickles to properly support union() and cartesian(): + writeStringAsPickle(str, dataOut) + case other => + throw new SparkException("Unexpected element type " + other.getClass) + } + } + + def writeToFile[T](items: java.util.Iterator[T], filename: String) { import scala.collection.JavaConverters._ - writeIteratorToPickleFile(items.asScala, filename) + writeToFile(items.asScala, filename) } - def writeIteratorToPickleFile[T](items: Iterator[T], filename: String) { + def writeToFile[T](items: Iterator[T], filename: String) { val file = new DataOutputStream(new FileOutputStream(filename)) for (item <- items) { - writeAsPickle(item, file) + writeToStream(item, file) } file.close() } @@ -300,10 +276,6 @@ private object Pickle { val TWO: Byte = 0x02.toByte val BINUNICODE: Byte = 'X' val STOP: Byte = '.' - val TUPLE2: Byte = 0x86.toByte - val EMPTY_LIST: Byte = ']' - val MARK: Byte = '(' - val APPENDS: Byte = 'e' } private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] { diff --git a/python/pyspark/context.py b/python/pyspark/context.py index a7ca8bc888..0fec1a6bf6 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -42,7 +42,7 @@ class SparkContext(object): _gateway = None _jvm = None - _writeIteratorToPickleFile = None + _writeToFile = None _takePartition = None _next_accum_id = 0 _active_spark_context = None @@ -125,8 +125,8 @@ class SparkContext(object): if not SparkContext._gateway: SparkContext._gateway = launch_gateway() SparkContext._jvm = SparkContext._gateway.jvm - SparkContext._writeIteratorToPickleFile = \ - SparkContext._jvm.PythonRDD.writeIteratorToPickleFile + SparkContext._writeToFile = \ + SparkContext._jvm.PythonRDD.writeToFile SparkContext._takePartition = \ SparkContext._jvm.PythonRDD.takePartition @@ -190,8 +190,8 @@ class SparkContext(object): for x in c: write_with_length(dump_pickle(x), tempFile) tempFile.close() - readRDDFromPickleFile = self._jvm.PythonRDD.readRDDFromPickleFile - jrdd = readRDDFromPickleFile(self._jsc, tempFile.name, numSlices) + readRDDFromFile = self._jvm.PythonRDD.readRDDFromFile + jrdd = readRDDFromFile(self._jsc, tempFile.name, numSlices) return RDD(jrdd, self) def textFile(self, name, minSplits=None): diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 7019fb8bee..d3c4d13a1e 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -54,6 +54,7 @@ class RDD(object): self.is_checkpointed = False self.ctx = ctx self._partitionFunc = None + self._stage_input_is_pairs = False @property def context(self): @@ -344,6 +345,7 @@ class RDD(object): yield pair else: yield pair + java_cartesian._stage_input_is_pairs = True return java_cartesian.flatMap(unpack_batches) def groupBy(self, f, numPartitions=None): @@ -391,8 +393,8 @@ class RDD(object): """ Return a list that contains all of the elements in this RDD. """ - picklesInJava = self._jrdd.collect().iterator() - return list(self._collect_iterator_through_file(picklesInJava)) + bytesInJava = self._jrdd.collect().iterator() + return list(self._collect_iterator_through_file(bytesInJava)) def _collect_iterator_through_file(self, iterator): # Transferring lots of data through Py4J can be slow because @@ -400,7 +402,7 @@ class RDD(object): # file and read it back. tempFile = NamedTemporaryFile(delete=False, dir=self.ctx._temp_dir) tempFile.close() - self.ctx._writeIteratorToPickleFile(iterator, tempFile.name) + self.ctx._writeToFile(iterator, tempFile.name) # Read the data into Python and deserialize it: with open(tempFile.name, 'rb') as tempFile: for item in read_from_pickle_file(tempFile): @@ -941,6 +943,7 @@ class PipelinedRDD(RDD): self.func = func self.preservesPartitioning = preservesPartitioning self._prev_jrdd = prev._jrdd + self._stage_input_is_pairs = prev._stage_input_is_pairs self.is_cached = False self.is_checkpointed = False self.ctx = prev.ctx @@ -959,7 +962,7 @@ class PipelinedRDD(RDD): def batched_func(split, iterator): return batched(oldfunc(split, iterator), batchSize) func = batched_func - cmds = [func, self._bypass_serializer] + cmds = [func, self._bypass_serializer, self._stage_input_is_pairs] pipe_command = ' '.join(b64enc(cloudpickle.dumps(f)) for f in cmds) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index fbc280fd37..fd02e1ee8f 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -93,6 +93,14 @@ def write_with_length(obj, stream): stream.write(obj) +def read_mutf8(stream): + """ + Read a string written with Java's DataOutputStream.writeUTF() method. + """ + length = struct.unpack('>H', stream.read(2))[0] + return stream.read(length).decode('utf8') + + def read_with_length(stream): length = read_int(stream) obj = stream.read(length) @@ -112,3 +120,13 @@ def read_from_pickle_file(stream): yield obj except EOFError: return + + +def read_pairs_from_pickle_file(stream): + try: + while True: + a = load_pickle(read_with_length(stream)) + b = load_pickle(read_with_length(stream)) + yield (a, b) + except EOFError: + return \ No newline at end of file diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 7696df9d1c..4e64557fc4 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -31,8 +31,8 @@ from pyspark.broadcast import Broadcast, _broadcastRegistry from pyspark.cloudpickle import CloudPickler from pyspark.files import SparkFiles from pyspark.serializers import write_with_length, read_with_length, write_int, \ - read_long, write_long, read_int, dump_pickle, load_pickle, read_from_pickle_file \ - SpecialLengths + read_long, write_long, read_int, dump_pickle, load_pickle, read_from_pickle_file, \ + SpecialLengths, read_mutf8, read_pairs_from_pickle_file def load_obj(infile): @@ -53,7 +53,7 @@ def main(infile, outfile): return # fetch name of workdir - spark_files_dir = load_pickle(read_with_length(infile)) + spark_files_dir = read_mutf8(infile) SparkFiles._root_directory = spark_files_dir SparkFiles._is_running_on_worker = True @@ -68,17 +68,21 @@ def main(infile, outfile): 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)))) + sys.path.append(os.path.join(spark_files_dir, read_mutf8(infile))) # now load function func = load_obj(infile) bypassSerializer = load_obj(infile) + stageInputIsPairs = load_obj(infile) if bypassSerializer: dumps = lambda x: x else: dumps = dump_pickle init_time = time.time() - iterator = read_from_pickle_file(infile) + if stageInputIsPairs: + iterator = read_pairs_from_pickle_file(infile) + else: + iterator = read_from_pickle_file(infile) try: for obj in func(split_index, iterator): write_with_length(dumps(obj), outfile) -- cgit v1.2.3 From cbb7f04aef2220ece93dea9f3fa98b5db5f270d6 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 5 Nov 2013 17:52:39 -0800 Subject: Add custom serializer support to PySpark. For now, this only adds MarshalSerializer, but it lays the groundwork for other supporting custom serializers. Many of these mechanisms can also be used to support deserialization of different data formats sent by Java, such as data encoded by MsgPack. This also fixes a bug in SparkContext.union(). --- .../org/apache/spark/api/python/PythonRDD.scala | 23 +- python/epydoc.conf | 2 +- python/pyspark/accumulators.py | 6 +- python/pyspark/context.py | 61 ++-- python/pyspark/rdd.py | 86 +++--- python/pyspark/serializers.py | 310 ++++++++++++++++----- python/pyspark/tests.py | 3 +- python/pyspark/worker.py | 41 ++- python/run-tests | 1 + 9 files changed, 363 insertions(+), 170 deletions(-) (limited to 'core/src') 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 eb0b0db0cc..ef9bf4db9b 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 @@ -221,18 +221,6 @@ private[spark] object PythonRDD { JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } - def writeStringAsPickle(elem: String, dOut: DataOutputStream) { - val s = elem.getBytes("UTF-8") - val length = 2 + 1 + 4 + s.length + 1 - dOut.writeInt(length) - dOut.writeByte(Pickle.PROTO) - dOut.writeByte(Pickle.TWO) - dOut.write(Pickle.BINUNICODE) - dOut.writeInt(Integer.reverseBytes(s.length)) - dOut.write(s) - dOut.writeByte(Pickle.STOP) - } - def writeToStream(elem: Any, dataOut: DataOutputStream) { elem match { case bytes: Array[Byte] => @@ -244,9 +232,7 @@ private[spark] object PythonRDD { dataOut.writeInt(pair._2.length) dataOut.write(pair._2) case str: String => - // Until we've implemented full custom serializer support, we need to return - // strings as Pickles to properly support union() and cartesian(): - writeStringAsPickle(str, dataOut) + dataOut.writeUTF(str) case other => throw new SparkException("Unexpected element type " + other.getClass) } @@ -271,13 +257,6 @@ private[spark] object PythonRDD { } } -private object Pickle { - val PROTO: Byte = 0x80.toByte - val TWO: Byte = 0x02.toByte - val BINUNICODE: Byte = 'X' - val STOP: Byte = '.' -} - 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/python/epydoc.conf b/python/epydoc.conf index 1d0d002d36..0b42e729f8 100644 --- a/python/epydoc.conf +++ b/python/epydoc.conf @@ -32,6 +32,6 @@ target: docs/ private: no -exclude: pyspark.cloudpickle pyspark.worker pyspark.join pyspark.serializers +exclude: pyspark.cloudpickle pyspark.worker pyspark.join pyspark.java_gateway pyspark.examples pyspark.shell pyspark.test pyspark.rddsampler pyspark.daemon diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index da3d96689a..2204e9c9ca 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -90,9 +90,11 @@ import struct import SocketServer import threading from pyspark.cloudpickle import CloudPickler -from pyspark.serializers import read_int, read_with_length, load_pickle +from pyspark.serializers import read_int, PickleSerializer +pickleSer = PickleSerializer() + # Holds accumulators registered on the current machine, keyed by ID. This is then used to send # the local accumulator updates back to the driver program at the end of a task. _accumulatorRegistry = {} @@ -211,7 +213,7 @@ class _UpdateRequestHandler(SocketServer.StreamRequestHandler): from pyspark.accumulators import _accumulatorRegistry num_updates = read_int(self.rfile) for _ in range(num_updates): - (aid, update) = load_pickle(read_with_length(self.rfile)) + (aid, update) = pickleSer._read_with_length(self.rfile) _accumulatorRegistry[aid] += update # Write a byte in acknowledgement self.wfile.write(struct.pack("!b", 1)) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 0fec1a6bf6..6bb1c6c3a1 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -26,7 +26,7 @@ from pyspark.accumulators import Accumulator from pyspark.broadcast import Broadcast from pyspark.files import SparkFiles from pyspark.java_gateway import launch_gateway -from pyspark.serializers import dump_pickle, write_with_length, batched +from pyspark.serializers import PickleSerializer, BatchedSerializer, MUTF8Deserializer from pyspark.storagelevel import StorageLevel from pyspark.rdd import RDD @@ -51,7 +51,7 @@ class SparkContext(object): def __init__(self, master, jobName, sparkHome=None, pyFiles=None, - environment=None, batchSize=1024): + environment=None, batchSize=1024, serializer=PickleSerializer()): """ Create a new SparkContext. @@ -67,6 +67,7 @@ class SparkContext(object): @param batchSize: The number of Python objects represented as a single Java object. Set 1 to disable batching or -1 to use an unlimited batch size. + @param serializer: The serializer for RDDs. >>> from pyspark.context import SparkContext @@ -83,7 +84,13 @@ class SparkContext(object): self.jobName = jobName self.sparkHome = sparkHome or None # None becomes null in Py4J self.environment = environment or {} - self.batchSize = batchSize # -1 represents a unlimited batch size + self._batchSize = batchSize # -1 represents an unlimited batch size + self._unbatched_serializer = serializer + if batchSize == 1: + self.serializer = self._unbatched_serializer + else: + self.serializer = BatchedSerializer(self._unbatched_serializer, + batchSize) # Create the Java SparkContext through Py4J empty_string_array = self._gateway.new_array(self._jvm.String, 0) @@ -184,15 +191,17 @@ class SparkContext(object): # 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) + 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) + serializer = BatchedSerializer(self._unbatched_serializer, + batchSize) + else: + serializer = self._unbatched_serializer + serializer.dump_stream(c, tempFile) tempFile.close() readRDDFromFile = self._jvm.PythonRDD.readRDDFromFile jrdd = readRDDFromFile(self._jsc, tempFile.name, numSlices) - return RDD(jrdd, self) + return RDD(jrdd, self, serializer) def textFile(self, name, minSplits=None): """ @@ -201,21 +210,39 @@ class SparkContext(object): RDD of Strings. """ minSplits = minSplits or min(self.defaultParallelism, 2) - jrdd = self._jsc.textFile(name, minSplits) - return RDD(jrdd, self) + return RDD(self._jsc.textFile(name, minSplits), self, + MUTF8Deserializer()) - def _checkpointFile(self, name): + def _checkpointFile(self, name, input_deserializer): jrdd = self._jsc.checkpointFile(name) - return RDD(jrdd, self) + return RDD(jrdd, self, input_deserializer) def union(self, rdds): """ Build the union of a list of RDDs. + + This supports unions() of RDDs with different serialized formats, + although this forces them to be reserialized using the default + serializer: + + >>> path = os.path.join(tempdir, "union-text.txt") + >>> with open(path, "w") as testFile: + ... testFile.write("Hello") + >>> textFile = sc.textFile(path) + >>> textFile.collect() + [u'Hello'] + >>> parallelized = sc.parallelize(["World!"]) + >>> sorted(sc.union([textFile, parallelized]).collect()) + [u'Hello', 'World!'] """ + first_jrdd_deserializer = rdds[0]._jrdd_deserializer + if any(x._jrdd_deserializer != first_jrdd_deserializer for x in rdds): + rdds = [x._reserialize() for x in rdds] first = rdds[0]._jrdd rest = [x._jrdd for x in rdds[1:]] - rest = ListConverter().convert(rest, self.gateway._gateway_client) - return RDD(self._jsc.union(first, rest), self) + rest = ListConverter().convert(rest, self._gateway._gateway_client) + return RDD(self._jsc.union(first, rest), self, + rdds[0]._jrdd_deserializer) def broadcast(self, value): """ @@ -223,7 +250,9 @@ class SparkContext(object): object for reading it in distributed functions. The variable will be sent to each cluster only once. """ - jbroadcast = self._jsc.broadcast(bytearray(dump_pickle(value))) + pickleSer = PickleSerializer() + pickled = pickleSer._dumps(value) + jbroadcast = self._jsc.broadcast(bytearray(pickled)) return Broadcast(jbroadcast.id(), value, jbroadcast, self._pickled_broadcast_vars) @@ -235,7 +264,7 @@ class SparkContext(object): and floating-point numbers if you do not provide one. For other types, a custom AccumulatorParam can be used. """ - if accum_param == None: + if accum_param is None: if isinstance(value, int): accum_param = accumulators.INT_ACCUMULATOR_PARAM elif isinstance(value, float): diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index d3c4d13a1e..6691c30519 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -18,7 +18,7 @@ from base64 import standard_b64encode as b64enc import copy from collections import defaultdict -from itertools import chain, ifilter, imap, product +from itertools import chain, ifilter, imap import operator import os import sys @@ -28,8 +28,8 @@ from tempfile import NamedTemporaryFile from threading import Thread from pyspark import cloudpickle -from pyspark.serializers import batched, Batch, dump_pickle, load_pickle, \ - read_from_pickle_file, pack_long +from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ + BatchedSerializer, pack_long from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup from pyspark.statcounter import StatCounter @@ -48,13 +48,12 @@ class RDD(object): operated on in parallel. """ - def __init__(self, jrdd, ctx): + def __init__(self, jrdd, ctx, jrdd_deserializer): self._jrdd = jrdd self.is_cached = False self.is_checkpointed = False self.ctx = ctx - self._partitionFunc = None - self._stage_input_is_pairs = False + self._jrdd_deserializer = jrdd_deserializer @property def context(self): @@ -248,7 +247,23 @@ class RDD(object): >>> rdd.union(rdd).collect() [1, 1, 2, 3, 1, 1, 2, 3] """ - return RDD(self._jrdd.union(other._jrdd), self.ctx) + if self._jrdd_deserializer == other._jrdd_deserializer: + rdd = RDD(self._jrdd.union(other._jrdd), self.ctx, + self._jrdd_deserializer) + return rdd + else: + # These RDDs contain data in different serialized formats, so we + # must normalize them to the default serializer. + self_copy = self._reserialize() + other_copy = other._reserialize() + return RDD(self_copy._jrdd.union(other_copy._jrdd), self.ctx, + self.ctx.serializer) + + def _reserialize(self): + if self._jrdd_deserializer == self.ctx.serializer: + return self + else: + return self.map(lambda x: x, preservesPartitioning=True) def __add__(self, other): """ @@ -335,18 +350,9 @@ class RDD(object): [(1, 1), (1, 2), (2, 1), (2, 2)] """ # Due to batching, we can't use the Java cartesian method. - java_cartesian = RDD(self._jrdd.cartesian(other._jrdd), self.ctx) - def unpack_batches(pair): - (x, y) = pair - if type(x) == Batch or type(y) == Batch: - xs = x.items if type(x) == Batch else [x] - ys = y.items if type(y) == Batch else [y] - for pair in product(xs, ys): - yield pair - else: - yield pair - java_cartesian._stage_input_is_pairs = True - return java_cartesian.flatMap(unpack_batches) + deserializer = CartesianDeserializer(self._jrdd_deserializer, + other._jrdd_deserializer) + return RDD(self._jrdd.cartesian(other._jrdd), self.ctx, deserializer) def groupBy(self, f, numPartitions=None): """ @@ -405,7 +411,7 @@ class RDD(object): self.ctx._writeToFile(iterator, tempFile.name) # Read the data into Python and deserialize it: with open(tempFile.name, 'rb') as tempFile: - for item in read_from_pickle_file(tempFile): + for item in self._jrdd_deserializer.load_stream(tempFile): yield item os.unlink(tempFile.name) @@ -573,7 +579,7 @@ class RDD(object): items = [] for partition in range(mapped._jrdd.splits().size()): iterator = self.ctx._takePartition(mapped._jrdd.rdd(), partition) - items.extend(self._collect_iterator_through_file(iterator)) + items.extend(mapped._collect_iterator_through_file(iterator)) if len(items) >= num: break return items[:num] @@ -737,6 +743,7 @@ class RDD(object): # Transferring O(n) objects to Java is too expensive. Instead, we'll # form the hash buckets in Python, transferring O(numPartitions) objects # to Java. Each object is a (splitNumber, [objects]) pair. + outputSerializer = self.ctx._unbatched_serializer def add_shuffle_key(split, iterator): buckets = defaultdict(list) @@ -745,14 +752,14 @@ class RDD(object): buckets[partitionFunc(k) % numPartitions].append((k, v)) for (split, items) in buckets.iteritems(): yield pack_long(split) - yield dump_pickle(Batch(items)) + yield outputSerializer._dumps(items) keyed = PipelinedRDD(self, add_shuffle_key) keyed._bypass_serializer = True pairRDD = self.ctx._jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD() partitioner = self.ctx._jvm.PythonPartitioner(numPartitions, id(partitionFunc)) jrdd = pairRDD.partitionBy(partitioner).values() - rdd = RDD(jrdd, self.ctx) + rdd = RDD(jrdd, self.ctx, BatchedSerializer(outputSerializer)) # This is required so that id(partitionFunc) remains unique, even if # partitionFunc is a lambda: rdd._partitionFunc = partitionFunc @@ -789,7 +796,8 @@ class RDD(object): numPartitions = self.ctx.defaultParallelism def combineLocally(iterator): combiners = {} - for (k, v) in iterator: + for x in iterator: + (k, v) = x if k not in combiners: combiners[k] = createCombiner(v) else: @@ -931,38 +939,38 @@ class PipelinedRDD(RDD): 20 """ def __init__(self, prev, func, preservesPartitioning=False): - if isinstance(prev, PipelinedRDD) and prev._is_pipelinable(): + if not isinstance(prev, PipelinedRDD) or not prev._is_pipelinable(): + # This transformation is the first in its stage: + self.func = func + self.preservesPartitioning = preservesPartitioning + self._prev_jrdd = prev._jrdd + self._prev_jrdd_deserializer = prev._jrdd_deserializer + else: prev_func = prev.func def pipeline_func(split, iterator): return func(split, prev_func(split, iterator)) self.func = pipeline_func self.preservesPartitioning = \ prev.preservesPartitioning and preservesPartitioning - self._prev_jrdd = prev._prev_jrdd - else: - self.func = func - self.preservesPartitioning = preservesPartitioning - self._prev_jrdd = prev._jrdd - self._stage_input_is_pairs = prev._stage_input_is_pairs + self._prev_jrdd = prev._prev_jrdd # maintain the pipeline + self._prev_jrdd_deserializer = prev._prev_jrdd_deserializer self.is_cached = False self.is_checkpointed = False self.ctx = prev.ctx self.prev = prev self._jrdd_val = None + self._jrdd_deserializer = self.ctx.serializer self._bypass_serializer = False @property def _jrdd(self): if self._jrdd_val: return self._jrdd_val - func = self.func - if not self._bypass_serializer and self.ctx.batchSize != 1: - oldfunc = self.func - batchSize = self.ctx.batchSize - def batched_func(split, iterator): - return batched(oldfunc(split, iterator), batchSize) - func = batched_func - cmds = [func, self._bypass_serializer, self._stage_input_is_pairs] + if self._bypass_serializer: + serializer = NoOpSerializer() + else: + serializer = self.ctx.serializer + cmds = [self.func, self._prev_jrdd_deserializer, serializer] pipe_command = ' '.join(b64enc(cloudpickle.dumps(f)) for f in cmds) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index fd02e1ee8f..4fb444443f 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -15,8 +15,58 @@ # limitations under the License. # -import struct +""" +PySpark supports custom serializers for transferring data; this can improve +performance. + +By default, PySpark uses L{PickleSerializer} to serialize objects using Python's +C{cPickle} serializer, which can serialize nearly any Python object. +Other serializers, like L{MarshalSerializer}, support fewer datatypes but can be +faster. + +The serializer is chosen when creating L{SparkContext}: + +>>> from pyspark.context import SparkContext +>>> from pyspark.serializers import MarshalSerializer +>>> sc = SparkContext('local', 'test', serializer=MarshalSerializer()) +>>> sc.parallelize(list(range(1000))).map(lambda x: 2 * x).take(10) +[0, 2, 4, 6, 8, 10, 12, 14, 16, 18] +>>> sc.stop() + +By default, PySpark serialize objects in batches; the batch size can be +controlled through SparkContext's C{batchSize} parameter +(the default size is 1024 objects): + +>>> sc = SparkContext('local', 'test', batchSize=2) +>>> rdd = sc.parallelize(range(16), 4).map(lambda x: x) + +Behind the scenes, this creates a JavaRDD with four partitions, each of +which contains two batches of two objects: + +>>> rdd.glom().collect() +[[0, 1, 2, 3], [4, 5, 6, 7], [8, 9, 10, 11], [12, 13, 14, 15]] +>>> rdd._jrdd.count() +8L +>>> sc.stop() + +A batch size of -1 uses an unlimited batch size, and a size of 1 disables +batching: + +>>> sc = SparkContext('local', 'test', batchSize=1) +>>> rdd = sc.parallelize(range(16), 4).map(lambda x: x) +>>> rdd.glom().collect() +[[0, 1, 2, 3], [4, 5, 6, 7], [8, 9, 10, 11], [12, 13, 14, 15]] +>>> rdd._jrdd.count() +16L +""" + import cPickle +from itertools import chain, izip, product +import marshal +import struct + + +__all__ = ["PickleSerializer", "MarshalSerializer"] class SpecialLengths(object): @@ -25,41 +75,206 @@ class SpecialLengths(object): TIMING_DATA = -3 -class Batch(object): +class Serializer(object): + + def dump_stream(self, iterator, stream): + """ + Serialize an iterator of objects to the output stream. + """ + raise NotImplementedError + + def load_stream(self, stream): + """ + Return an iterator of deserialized objects from the input stream. + """ + raise NotImplementedError + + + def _load_stream_without_unbatching(self, stream): + return self.load_stream(stream) + + # Note: our notion of "equality" is that output generated by + # equal serializers can be deserialized using the same serializer. + + # This default implementation handles the simple cases; + # subclasses should override __eq__ as appropriate. + + def __eq__(self, other): + return isinstance(other, self.__class__) + + def __ne__(self, other): + return not self.__eq__(other) + + +class FramedSerializer(Serializer): + """ + Serializer that writes objects as a stream of (length, data) pairs, + where C{length} is a 32-bit integer and data is C{length} bytes. + """ + + def dump_stream(self, iterator, stream): + for obj in iterator: + self._write_with_length(obj, stream) + + def load_stream(self, stream): + while True: + try: + yield self._read_with_length(stream) + except EOFError: + return + + def _write_with_length(self, obj, stream): + serialized = self._dumps(obj) + write_int(len(serialized), stream) + stream.write(serialized) + + def _read_with_length(self, stream): + length = read_int(stream) + obj = stream.read(length) + if obj == "": + raise EOFError + return self._loads(obj) + + def _dumps(self, obj): + """ + Serialize an object into a byte array. + When batching is used, this will be called with an array of objects. + """ + raise NotImplementedError + + def _loads(self, obj): + """ + Deserialize an object from a byte array. + """ + raise NotImplementedError + + +class BatchedSerializer(Serializer): + """ + Serializes a stream of objects in batches by calling its wrapped + Serializer with streams of objects. + """ + + UNLIMITED_BATCH_SIZE = -1 + + def __init__(self, serializer, batchSize=UNLIMITED_BATCH_SIZE): + self.serializer = serializer + self.batchSize = batchSize + + def _batched(self, iterator): + if self.batchSize == self.UNLIMITED_BATCH_SIZE: + yield list(iterator) + else: + items = [] + count = 0 + for item in iterator: + items.append(item) + count += 1 + if count == self.batchSize: + yield items + items = [] + count = 0 + if items: + yield items + + def dump_stream(self, iterator, stream): + if isinstance(iterator, basestring): + iterator = [iterator] + self.serializer.dump_stream(self._batched(iterator), stream) + + def load_stream(self, stream): + return chain.from_iterable(self._load_stream_without_unbatching(stream)) + + def _load_stream_without_unbatching(self, stream): + return self.serializer.load_stream(stream) + + def __eq__(self, other): + return isinstance(other, BatchedSerializer) and \ + other.serializer == self.serializer + + def __str__(self): + return "BatchedSerializer<%s>" % str(self.serializer) + + +class CartesianDeserializer(FramedSerializer): """ - Used to store multiple RDD entries as a single Java object. + Deserializes the JavaRDD cartesian() of two PythonRDDs. + """ + + def __init__(self, key_ser, val_ser): + self.key_ser = key_ser + self.val_ser = val_ser + + def load_stream(self, stream): + key_stream = self.key_ser._load_stream_without_unbatching(stream) + val_stream = self.val_ser._load_stream_without_unbatching(stream) + key_is_batched = isinstance(self.key_ser, BatchedSerializer) + val_is_batched = isinstance(self.val_ser, BatchedSerializer) + for (keys, vals) in izip(key_stream, val_stream): + keys = keys if key_is_batched else [keys] + vals = vals if val_is_batched else [vals] + for pair in product(keys, vals): + yield pair + + def __eq__(self, other): + return isinstance(other, CartesianDeserializer) and \ + self.key_ser == other.key_ser and self.val_ser == other.val_ser + + def __str__(self): + return "CartesianDeserializer<%s, %s>" % \ + (str(self.key_ser), str(self.val_ser)) + + +class NoOpSerializer(FramedSerializer): + + def _loads(self, obj): return obj + def _dumps(self, obj): return obj + + +class PickleSerializer(FramedSerializer): + """ + Serializes objects using Python's cPickle serializer: + + http://docs.python.org/2/library/pickle.html + + This serializer supports nearly any Python object, but may + not be as fast as more specialized serializers. + """ + + def _dumps(self, obj): return cPickle.dumps(obj, 2) + _loads = cPickle.loads + - This relieves us from having to explicitly track whether an RDD - is stored as batches of objects and avoids problems when processing - the union() of batched and unbatched RDDs (e.g. the union() of textFile() - with another RDD). +class MarshalSerializer(FramedSerializer): """ - def __init__(self, items): - self.items = items + Serializes objects using Python's Marshal serializer: + http://docs.python.org/2/library/marshal.html -def batched(iterator, batchSize): - if batchSize == -1: # unlimited batch size - yield Batch(list(iterator)) - else: - items = [] - count = 0 - for item in iterator: - items.append(item) - count += 1 - if count == batchSize: - yield Batch(items) - items = [] - count = 0 - if items: - yield Batch(items) + This serializer is faster than PickleSerializer but supports fewer datatypes. + """ + + _dumps = marshal.dumps + _loads = marshal.loads -def dump_pickle(obj): - return cPickle.dumps(obj, 2) +class MUTF8Deserializer(Serializer): + """ + Deserializes streams written by Java's DataOutputStream.writeUTF(). + """ + def _loads(self, stream): + length = struct.unpack('>H', stream.read(2))[0] + return stream.read(length).decode('utf8') -load_pickle = cPickle.loads + def load_stream(self, stream): + while True: + try: + yield self._loads(stream) + except struct.error: + return + except EOFError: + return def read_long(stream): @@ -90,43 +305,4 @@ def write_int(value, stream): def write_with_length(obj, stream): write_int(len(obj), stream) - stream.write(obj) - - -def read_mutf8(stream): - """ - Read a string written with Java's DataOutputStream.writeUTF() method. - """ - length = struct.unpack('>H', stream.read(2))[0] - return stream.read(length).decode('utf8') - - -def read_with_length(stream): - length = read_int(stream) - obj = stream.read(length) - if obj == "": - raise EOFError - return obj - - -def read_from_pickle_file(stream): - try: - while True: - obj = load_pickle(read_with_length(stream)) - if type(obj) == Batch: # We don't care about inheritance - for item in obj.items: - yield item - else: - yield obj - except EOFError: - return - - -def read_pairs_from_pickle_file(stream): - try: - while True: - a = load_pickle(read_with_length(stream)) - b = load_pickle(read_with_length(stream)) - yield (a, b) - except EOFError: - return \ No newline at end of file + stream.write(obj) \ No newline at end of file diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 29d6a128f6..621e1cb58c 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -86,7 +86,8 @@ class TestCheckpoint(PySparkTestCase): time.sleep(1) # 1 second self.assertTrue(flatMappedRDD.getCheckpointFile() is not None) - recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile()) + recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile(), + flatMappedRDD._jrdd_deserializer) self.assertEquals([1, 2, 3, 4], recovered.collect()) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 4e64557fc4..5b16d5db7e 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -30,13 +30,17 @@ from pyspark.accumulators import _accumulatorRegistry from pyspark.broadcast import Broadcast, _broadcastRegistry from pyspark.cloudpickle import CloudPickler from pyspark.files import SparkFiles -from pyspark.serializers import write_with_length, read_with_length, write_int, \ - read_long, write_long, read_int, dump_pickle, load_pickle, read_from_pickle_file, \ - SpecialLengths, read_mutf8, read_pairs_from_pickle_file +from pyspark.serializers import write_with_length, write_int, read_long, \ + write_long, read_int, SpecialLengths, MUTF8Deserializer, PickleSerializer + + +pickleSer = PickleSerializer() +mutf8_deserializer = MUTF8Deserializer() def load_obj(infile): - return load_pickle(standard_b64decode(infile.readline().strip())) + decoded = standard_b64decode(infile.readline().strip()) + return pickleSer._loads(decoded) def report_times(outfile, boot, init, finish): @@ -53,7 +57,7 @@ def main(infile, outfile): return # fetch name of workdir - spark_files_dir = read_mutf8(infile) + spark_files_dir = mutf8_deserializer._loads(infile) SparkFiles._root_directory = spark_files_dir SparkFiles._is_running_on_worker = True @@ -61,31 +65,24 @@ def main(infile, outfile): 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)) + value = pickleSer._read_with_length(infile) + _broadcastRegistry[bid] = Broadcast(bid, 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, read_mutf8(infile))) + filename = mutf8_deserializer._loads(infile) + sys.path.append(os.path.join(spark_files_dir, filename)) - # now load function + # Load this stage's function and serializer: func = load_obj(infile) - bypassSerializer = load_obj(infile) - stageInputIsPairs = load_obj(infile) - if bypassSerializer: - dumps = lambda x: x - else: - dumps = dump_pickle + deserializer = load_obj(infile) + serializer = load_obj(infile) init_time = time.time() - if stageInputIsPairs: - iterator = read_pairs_from_pickle_file(infile) - else: - iterator = read_from_pickle_file(infile) try: - for obj in func(split_index, iterator): - write_with_length(dumps(obj), outfile) + iterator = deserializer.load_stream(infile) + serializer.dump_stream(func(split_index, iterator), outfile) except Exception as e: write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) write_with_length(traceback.format_exc(), outfile) @@ -96,7 +93,7 @@ def main(infile, outfile): write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) write_int(len(_accumulatorRegistry), outfile) for (aid, accum) in _accumulatorRegistry.items(): - write_with_length(dump_pickle((aid, accum._value)), outfile) + pickleSer._write_with_length((aid, accum._value), outfile) if __name__ == '__main__': diff --git a/python/run-tests b/python/run-tests index cbc554ea9d..d4dad672d2 100755 --- a/python/run-tests +++ b/python/run-tests @@ -37,6 +37,7 @@ 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 "-m doctest pyspark/serializers.py" run_test "pyspark/tests.py" if [[ $FAILED != 0 ]]; then -- cgit v1.2.3 From ffa5bedf46fbc89ad5c5658f3b423dfff49b70f0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 10 Nov 2013 12:58:28 -0800 Subject: Send PySpark commands as bytes insetad of strings. --- .../org/apache/spark/api/python/PythonRDD.scala | 24 ++++------------------ python/pyspark/rdd.py | 12 +++++------ python/pyspark/serializers.py | 5 +++++ python/pyspark/worker.py | 12 ++--------- 4 files changed, 17 insertions(+), 36 deletions(-) (limited to 'core/src') 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 ef9bf4db9b..132e4fb0d2 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 @@ -27,13 +27,12 @@ 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]( parent: RDD[T], - command: Seq[String], + command: Array[Byte], envVars: JMap[String, String], pythonIncludes: JList[String], preservePartitoning: Boolean, @@ -44,21 +43,10 @@ private[spark] class PythonRDD[T: ClassManifest]( 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], - pythonIncludes: JList[String], - preservePartitoning: Boolean, pythonExec: String, - broadcastVars: JList[Broadcast[Array[Byte]]], - accumulator: Accumulator[JList[Array[Byte]]]) = - this(parent, PipedRDD.tokenize(command), envVars, pythonIncludes, preservePartitoning, pythonExec, - broadcastVars, accumulator) - override def getPartitions = parent.partitions override val partitioner = if (preservePartitoning) parent.partitioner else None - override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { val startTime = System.currentTimeMillis val env = SparkEnv.get @@ -71,7 +59,6 @@ private[spark] class PythonRDD[T: ClassManifest]( 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 @@ -87,17 +74,14 @@ private[spark] class PythonRDD[T: ClassManifest]( dataOut.writeInt(pythonIncludes.length) pythonIncludes.foreach(dataOut.writeUTF) dataOut.flush() - // Serialized user code - for (elem <- command) { - printOut.println(elem) - } - printOut.flush() + // Serialized command: + dataOut.writeInt(command.length) + dataOut.write(command) // Data values for (elem <- parent.iterator(split, context)) { PythonRDD.writeToStream(elem, dataOut) } dataOut.flush() - printOut.flush() worker.shutdownOutput() } catch { case e: IOException => diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6691c30519..062f44f81e 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -27,9 +27,8 @@ from subprocess import Popen, PIPE from tempfile import NamedTemporaryFile from threading import Thread -from pyspark import cloudpickle from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ - BatchedSerializer, pack_long + BatchedSerializer, CloudPickleSerializer, pack_long from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup from pyspark.statcounter import StatCounter @@ -970,8 +969,8 @@ class PipelinedRDD(RDD): serializer = NoOpSerializer() else: serializer = self.ctx.serializer - cmds = [self.func, self._prev_jrdd_deserializer, serializer] - pipe_command = ' '.join(b64enc(cloudpickle.dumps(f)) for f in cmds) + command = (self.func, self._prev_jrdd_deserializer, serializer) + pickled_command = CloudPickleSerializer()._dumps(command) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], self.ctx._gateway._gateway_client) @@ -982,8 +981,9 @@ class PipelinedRDD(RDD): 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, includes, self.preservesPartitioning, self.ctx.pythonExec, - broadcast_vars, self.ctx._javaAccumulator, class_manifest) + bytearray(pickled_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/serializers.py b/python/pyspark/serializers.py index 4fb444443f..b23804b33c 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -64,6 +64,7 @@ import cPickle from itertools import chain, izip, product import marshal import struct +from pyspark import cloudpickle __all__ = ["PickleSerializer", "MarshalSerializer"] @@ -244,6 +245,10 @@ class PickleSerializer(FramedSerializer): def _dumps(self, obj): return cPickle.dumps(obj, 2) _loads = cPickle.loads +class CloudPickleSerializer(PickleSerializer): + + def _dumps(self, obj): return cloudpickle.dumps(obj, 2) + class MarshalSerializer(FramedSerializer): """ diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 5b16d5db7e..2751f1239e 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -23,7 +23,6 @@ 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 # copy_reg module. from pyspark.accumulators import _accumulatorRegistry @@ -38,11 +37,6 @@ pickleSer = PickleSerializer() mutf8_deserializer = MUTF8Deserializer() -def load_obj(infile): - decoded = standard_b64decode(infile.readline().strip()) - return pickleSer._loads(decoded) - - def report_times(outfile, boot, init, finish): write_int(SpecialLengths.TIMING_DATA, outfile) write_long(1000 * boot, outfile) @@ -75,10 +69,8 @@ def main(infile, outfile): filename = mutf8_deserializer._loads(infile) sys.path.append(os.path.join(spark_files_dir, filename)) - # Load this stage's function and serializer: - func = load_obj(infile) - deserializer = load_obj(infile) - serializer = load_obj(infile) + command = pickleSer._read_with_length(infile) + (func, deserializer, serializer) = command init_time = time.time() try: iterator = deserializer.load_stream(infile) -- cgit v1.2.3 From a8bfdd4377918bf665d4615c7b45ed84a7a9cebc Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 12 Nov 2013 11:38:20 +0530 Subject: Enabled remote death watch and a way to configure the timeouts for akka heartbeats. --- .../spark/executor/StandaloneExecutorBackend.scala | 4 +- .../cluster/StandaloneSchedulerBackend.scala | 5 --- .../scala/org/apache/spark/util/AkkaUtils.scala | 44 ++++++++++++---------- 3 files changed, 27 insertions(+), 26 deletions(-) (limited to 'core/src') 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 f705a5631a..a76a8e9730 100644 --- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala @@ -59,12 +59,12 @@ private[spark] class StandaloneExecutorBackend( driver = context.actorSelection(driverUrl) driver ! RegisterExecutor(executorId, hostPort, cores) context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - // context.watch(driver) // Doesn't work with remote actors, but useful for testing } override def receive = { case RegisteredExecutor(sparkProperties) => logInfo("Successfully registered with driver") + context.watch(sender) //Start watching for terminated messages. // Make this host instead of hostPort ? executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties) @@ -81,7 +81,7 @@ private[spark] class StandaloneExecutorBackend( executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask) } - case DisassociatedEvent(_, _, _) => + case Terminated(actor) => logError("Driver terminated or disconnected! Shutting down.") System.exit(1) } 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 b6f0ec961a..2d09b329f2 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 @@ -102,11 +102,6 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor case Terminated(actor) => actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated")) - case DisassociatedEvent(_, remoteAddress, _) => - addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client disconnected")) - - case AssociationErrorEvent(_, _, remoteAddress, _) => - addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client shutdown")) } // Make fake resource offers on all executors diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 8daf50ab69..2a831382df 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -43,29 +43,35 @@ private[spark] object AkkaUtils { val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - // 10 seconds is the default akka timeout, but in a cluster, we need higher by default. - val akkaWriteTimeout = System.getProperty("spark.akka.writeTimeout", "30").toInt - val akkaConf = ConfigFactory.parseString(""" - akka.daemonic = on - akka.loggers = [""akka.event.slf4j.Slf4jLogger""] - akka.stdout-loglevel = "ERROR" - akka.actor.provider = "akka.remote.RemoteActorRefProvider" - akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" - akka.remote.netty.tcp.hostname = "%s" - akka.remote.netty.tcp.port = %d - akka.remote.netty.tcp.connection-timeout = %d s - akka.remote.netty.tcp.maximum-frame-size = %dMiB - akka.remote.netty.tcp.execution-pool-size = %d - akka.actor.default-dispatcher.throughput = %d - akka.remote.log-remote-lifecycle-events = %s - """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize, - lifecycleEvents)) + val akkaHeartBeatPauses = System.getProperty("spark.akka.pauses", "30").toInt + val akkaFailureDetector = System.getProperty("spark.akka.failure-detector.threshold", "30").toInt + val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "3").toInt + + val akkaConf = ConfigFactory.parseString( + s""" + |akka.daemonic = on + |akka.loggers = [""akka.event.slf4j.Slf4jLogger""] + |akka.stdout-loglevel = "ERROR" + |akka.remote.watch-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s + |akka.remote.watch-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s + |akka.remote.watch-failure-detector.threshold = $akkaFailureDetector + |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s + |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s + |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector + |akka.actor.provider = "akka.remote.RemoteActorRefProvider" + |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" + |akka.remote.netty.tcp.hostname = "$host" + |akka.remote.netty.tcp.port = $port + |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s + |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}MiB + |akka.remote.netty.tcp.execution-pool-size = $akkaThreads + |akka.actor.default-dispatcher.throughput = $akkaBatchSize + |akka.remote.log-remote-lifecycle-events = $lifecycleEvents + """.stripMargin) val actorSystem = ActorSystem(name, akkaConf) - // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a - // hack because Akka doesn't let you figure out the port through the public API yet. val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.getDefaultAddress.port.get (actorSystem, boundPort) -- cgit v1.2.3 From 6860b79f6e4cc0d38b08848f19127c259d9b5069 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 12 Nov 2013 12:43:53 +0530 Subject: Remove deprecated actorFor and use actorSelection everywhere. --- .../scala/org/apache/spark/MapOutputTracker.scala | 8 +++++-- .../src/main/scala/org/apache/spark/SparkEnv.scala | 8 +++---- .../org/apache/spark/deploy/client/Client.scala | 23 +++++--------------- .../org/apache/spark/deploy/worker/Worker.scala | 23 ++++++++++++++------ .../apache/spark/storage/BlockManagerMaster.scala | 25 +++++++++++++--------- .../org/apache/spark/storage/ThreadingTest.scala | 2 +- .../org/apache/spark/MapOutputTrackerSuite.scala | 14 ++++++------ .../apache/spark/storage/BlockManagerSuite.scala | 2 +- .../spark/streaming/examples/ActorWordCount.scala | 2 +- .../streaming/dstream/NetworkInputDStream.scala | 2 +- 10 files changed, 58 insertions(+), 51 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 1afb1870f1..6590e9779e 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -60,7 +60,7 @@ private[spark] class MapOutputTracker extends Logging { private val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") // Set to the MapOutputTrackerActor living on the driver - var trackerActor: ActorRef = _ + var trackerActor: Either[ActorRef, ActorSelection] = _ private var mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] @@ -79,7 +79,11 @@ private[spark] class MapOutputTracker extends Logging { // throw a SparkException if this fails. def askTracker(message: Any): Any = { try { - val future = trackerActor.ask(message)(timeout) + val future = if (trackerActor.isLeft ) { + trackerActor.left.get.ask(message)(timeout) + } else { + trackerActor.right.get.ask(message)(timeout) + } return Await.result(future, timeout) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index a267407c67..0d9bd500e4 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -20,7 +20,7 @@ package org.apache.spark import collection.mutable import serializer.Serializer -import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} +import akka.actor._ import akka.remote.RemoteActorRefProvider import org.apache.spark.broadcast.BroadcastManager @@ -161,17 +161,17 @@ object SparkEnv extends Logging { val closureSerializer = serializerManager.get( System.getProperty("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")) - def registerOrLookup(name: String, newActor: => Actor): ActorRef = { + def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = { if (isDriver) { logInfo("Registering " + name) - actorSystem.actorOf(Props(newActor), name = name) + Left(actorSystem.actorOf(Props(newActor), name = name)) } else { val driverHost: String = System.getProperty("spark.driver.host", "localhost") val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt Utils.checkHost(driverHost, "Expected hostname") val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name) logInfo("Connecting to " + name + ": " + url) - actorSystem.actorFor(url) + Right(actorSystem.actorSelection(url)) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 164386782c..000d1ee9f8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -49,18 +49,14 @@ private[spark] class Client( var appId: String = null class ClientActor extends Actor with Logging { - var master: ActorRef = null - var masterAddress: Address = null + var master: ActorSelection = null var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times override def preStart() { logInfo("Connecting to master " + masterUrl) try { - master = context.actorFor(Master.toAkkaUrl(masterUrl)) - masterAddress = master.path.address + master = context.actorSelection(Master.toAkkaUrl(masterUrl)) master ! RegisterApplication(appDescription) - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - context.watch(master) // Doesn't work with remote actors, but useful for testing } catch { case e: Exception => logError("Failed to connect to master", e) @@ -71,6 +67,7 @@ private[spark] class Client( override def receive = { case RegisteredApplication(appId_) => + context.watch(sender) appId = appId_ listener.connected(appId) @@ -92,18 +89,8 @@ private[spark] class Client( listener.executorRemoved(fullId, message.getOrElse(""), exitStatus) } - case Terminated(actor_) if actor_ == master => - logError("Connection to master failed; stopping client") - markDisconnected() - context.stop(self) - - case DisassociatedEvent(_, address, _) if address == masterAddress => - logError("Connection to master failed; stopping client") - markDisconnected() - context.stop(self) - - case AssociationErrorEvent(_, _, address, _) if address == masterAddress => - logError("Connection to master failed; stopping client") + case Terminated(actor_) => + logError(s"Connection to $actor_ dropped, stopping client") markDisconnected() context.stop(self) 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 3904b701b2..400d6f26ea 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 @@ -24,7 +24,7 @@ import java.io.File import scala.collection.mutable.HashMap import scala.concurrent.duration._ -import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} +import akka.actor._ import akka.remote.{RemotingLifecycleEvent, AssociationErrorEvent, DisassociatedEvent} import org.apache.spark.Logging @@ -34,6 +34,16 @@ 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.{Utils, AkkaUtils} +import org.apache.spark.deploy.DeployMessages.WorkerStateResponse +import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed +import org.apache.spark.deploy.DeployMessages.KillExecutor +import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import scala.Some +import org.apache.spark.deploy.DeployMessages.Heartbeat +import org.apache.spark.deploy.DeployMessages.RegisteredWorker +import akka.remote.DisassociatedEvent +import org.apache.spark.deploy.DeployMessages.LaunchExecutor +import org.apache.spark.deploy.DeployMessages.RegisterWorker private[spark] class Worker( @@ -54,7 +64,7 @@ private[spark] class Worker( // Send a heartbeat every (heartbeat timeout) / 4 milliseconds val HEARTBEAT_MILLIS = System.getProperty("spark.worker.timeout", "60").toLong * 1000 / 4 - var master: ActorRef = null + var master: ActorSelection = null var masterWebUiUrl : String = "" val workerId = generateWorkerId() var sparkHome: File = null @@ -111,10 +121,8 @@ private[spark] class Worker( def connectToMaster() { logInfo("Connecting to master " + masterUrl) - master = context.actorFor(Master.toAkkaUrl(masterUrl)) + master = context.actorSelection(Master.toAkkaUrl(masterUrl)) master ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get, publicAddress) - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - context.watch(master) // Doesn't work with remote actors, but useful for testing } import context.dispatcher @@ -123,6 +131,8 @@ private[spark] class Worker( case RegisteredWorker(url) => masterWebUiUrl = url logInfo("Successfully registered with master") + context.watch(sender) // remote death watch for master + //TODO: Is heartbeat really necessary akka does it anyway ! context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis) { master ! Heartbeat(workerId) } @@ -165,7 +175,8 @@ private[spark] class Worker( logInfo("Asked to kill unknown executor " + fullId) } - case DisassociatedEvent(_, _, _) => + case Terminated(actor_) => + logInfo(s"$actor_ terminated !") masterDisconnected() case RequestWorkerState => { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 0c977f05d1..c1aa43d59c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -17,14 +17,7 @@ package org.apache.spark.storage -import java.io._ -import java.util.{HashMap => JHashMap} - -import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.util.Random - -import akka.actor.{Actor, ActorRef, ActorSystem, Props} +import akka.actor._ import scala.concurrent.Await import scala.concurrent.Future import scala.concurrent.ExecutionContext.Implicits.global @@ -34,8 +27,16 @@ import scala.concurrent.duration._ import org.apache.spark.{Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ +import org.apache.spark.storage.BlockManagerMessages.GetLocations +import org.apache.spark.storage.BlockManagerMessages.GetLocationsMultipleBlockIds +import org.apache.spark.storage.BlockManagerMessages.RegisterBlockManager +import org.apache.spark.storage.BlockManagerMessages.HeartBeat +import org.apache.spark.storage.BlockManagerMessages.RemoveExecutor +import org.apache.spark.storage.BlockManagerMessages.GetPeers +import org.apache.spark.storage.BlockManagerMessages.RemoveBlock +import org.apache.spark.storage.BlockManagerMessages.RemoveRdd -private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging { +private[spark] class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection]) extends Logging { val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt @@ -165,7 +166,11 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi while (attempts < AKKA_RETRY_ATTEMPTS) { attempts += 1 try { - val future = driverActor.ask(message)(timeout) + val future = if (driverActor.isLeft ) { + driverActor.left.get.ask(message)(timeout) + } else { + driverActor.right.get.ask(message)(timeout) + } val result = Await.result(future, timeout) if (result == null) { throw new SparkException("BlockManagerMaster returned null") 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 f2ae8dd97d..1e6da269f2 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -93,7 +93,7 @@ private[spark] object ThreadingTest { val actorSystem = ActorSystem("test") val serializer = new KryoSerializer val blockManagerMaster = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))) + Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 18fb1bf590..955f6cdadc 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -49,14 +49,14 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master start and stop") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTracker() - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) + tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker)))) tracker.stop() } test("master register and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTracker() - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) + tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker)))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -75,7 +75,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master register and unregister and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTracker() - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) + tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker)))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -103,13 +103,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { System.setProperty("spark.hostPort", hostname + ":" + boundPort) val masterTracker = new MapOutputTracker() - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker") + masterTracker.trackerActor = Left(actorSystem.actorOf( + Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker")) val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0) val slaveTracker = new MapOutputTracker() - slaveTracker.trackerActor = slaveSystem.actorFor( - "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker") + slaveTracker.trackerActor = Right(slaveSystem.actorSelection( + "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker")) masterTracker.registerShuffle(10, 1) masterTracker.incrementEpoch() 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 038a9acb85..4fdc43cc22 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -53,7 +53,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT System.setProperty("spark.hostPort", "localhost:" + boundPort) master = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))) + Left(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") diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 08e399f9ee..128711aacd 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -86,7 +86,7 @@ class FeederActor extends Actor { class SampleActorReceiver[T: ClassTag](urlOfPublisher: String) extends Actor with Receiver { - lazy private val remotePublisher = context.actorFor(urlOfPublisher) + lazy private val remotePublisher = context.actorSelection(urlOfPublisher) override def preStart = remotePublisher ! SubscribeReceiver(context.self) 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 394a39fbb0..b2f9f8b224 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 @@ -178,7 +178,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging val ip = System.getProperty("spark.driver.host", "localhost") val port = System.getProperty("spark.driver.port", "7077").toInt val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) - val tracker = env.actorSystem.actorFor(url) + val tracker = env.actorSystem.actorSelection(url) val timeout = 5.seconds override def preStart() { -- cgit v1.2.3 From 68e5ad58b7e7e3e1b42852de8d0fdf9e9b9c1a14 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Wed, 13 Nov 2013 14:32:50 -0800 Subject: Extracted TaskScheduler interface. Also changed the default maximum number of task failures to be 0 when running in local mode. --- .../main/scala/org/apache/spark/SparkContext.scala | 14 +- .../apache/spark/scheduler/ClusterScheduler.scala | 493 +++++++++++++++++++++ .../apache/spark/scheduler/TaskResultGetter.scala | 2 +- .../org/apache/spark/scheduler/TaskScheduler.scala | 492 -------------------- .../apache/spark/scheduler/TaskSetManager.scala | 17 +- .../cluster/CoarseGrainedSchedulerBackend.scala | 4 +- .../scheduler/cluster/SimrSchedulerBackend.scala | 4 +- .../cluster/SparkDeploySchedulerBackend.scala | 4 +- .../mesos/CoarseMesosSchedulerBackend.scala | 4 +- .../cluster/mesos/MesosSchedulerBackend.scala | 4 +- .../spark/scheduler/local/LocalBackend.scala | 9 +- .../spark/scheduler/ClusterSchedulerSuite.scala | 265 +++++++++++ .../apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../spark/scheduler/TaskResultGetterSuite.scala | 6 +- .../spark/scheduler/TaskSchedulerSuite.scala | 265 ----------- .../spark/scheduler/TaskSetManagerSuite.scala | 21 +- 16 files changed, 806 insertions(+), 800 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerSuite.scala (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1850436ff2..e8ff4da475 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -159,26 +159,26 @@ class SparkContext( master match { case "local" => - val scheduler = new TaskScheduler(this) + val scheduler = new ClusterScheduler(this, isLocal = true) val backend = new LocalBackend(scheduler, 1) scheduler.initialize(backend) scheduler case LOCAL_N_REGEX(threads) => - val scheduler = new TaskScheduler(this) + val scheduler = new ClusterScheduler(this, isLocal = true) val backend = new LocalBackend(scheduler, threads.toInt) scheduler.initialize(backend) scheduler case SPARK_REGEX(sparkUrl) => - val scheduler = new TaskScheduler(this) + val scheduler = new ClusterScheduler(this) val masterUrls = sparkUrl.split(",").map("spark://" + _) val backend = new SparkDeploySchedulerBackend(scheduler, this, masterUrls, appName) scheduler.initialize(backend) scheduler case SIMR_REGEX(simrUrl) => - val scheduler = new TaskScheduler(this) + val scheduler = new ClusterScheduler(this) val backend = new SimrSchedulerBackend(scheduler, this, simrUrl) scheduler.initialize(backend) scheduler @@ -192,7 +192,7 @@ class SparkContext( memoryPerSlaveInt, SparkContext.executorMemoryRequested)) } - val scheduler = new TaskScheduler(this) + val scheduler = new ClusterScheduler(this, isLocal = true) val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val masterUrls = localCluster.start() @@ -207,7 +207,7 @@ class SparkContext( val scheduler = try { val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler") val cons = clazz.getConstructor(classOf[SparkContext]) - cons.newInstance(this).asInstanceOf[TaskScheduler] + cons.newInstance(this).asInstanceOf[ClusterScheduler] } catch { // TODO: Enumerate the exact reasons why it can fail // But irrespective of it, it means we cannot proceed ! @@ -221,7 +221,7 @@ class SparkContext( case MESOS_REGEX(mesosUrl) => MesosNativeLibrary.load() - val scheduler = new TaskScheduler(this) + val scheduler = new ClusterScheduler(this) val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean val backend = if (coarseGrained) { new CoarseMesosSchedulerBackend(scheduler, this, mesosUrl, appName) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala new file mode 100644 index 0000000000..c7d1295215 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala @@ -0,0 +1,493 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicLong +import java.util.{TimerTask, Timer} + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet + +import org.apache.spark._ +import org.apache.spark.TaskState.TaskState +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode + +/** + * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. + * It can also work with a local setup by using a LocalBackend and setting isLocal to true. + * It handles common logic, like determining a scheduling order across jobs, waking up to launch + * speculative tasks, etc. + * + * Clients should first call initialize() and start(), then submit task sets through the + * runTasks method. + * + * 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, isLocal: Boolean = false) + extends TaskScheduler 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 + + // TaskSetManagers are not thread safe, so any access to one should be synchronized + // on this class. + val activeTaskSets = new HashMap[String, TaskSetManager] + + val MAX_TASK_FAILURES = { + if (isLocal) { + // No sense in retrying if all tasks run locally! + 0 + } else { + System.getProperty("spark.task.maxFailures", "4").toInt + } + } + + val taskIdToTaskSetId = new HashMap[Long, String] + val taskIdToExecutorId = new HashMap[Long, String] + val taskSetTaskIds = new HashMap[String, HashSet[Long]] + + @volatile private var hasReceivedTask = false + @volatile private var hasLaunchedTask = false + private val starvationTimer = new Timer(true) + + // Incrementing task IDs + val nextTaskId = new AtomicLong(0) + + // Which executor IDs we have executors on + val activeExecutorIds = new 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 executorsByHost = new HashMap[String, HashSet[String]] + + private val executorIdToHost = new HashMap[String, String] + + // Listener object to pass upcalls into + var dagScheduler: DAGScheduler = null + + var backend: SchedulerBackend = null + + val mapOutputTracker = SparkEnv.get.mapOutputTracker + + var schedulableBuilder: SchedulableBuilder = null + var rootPool: Pool = null + // default scheduler is FIFO + val schedulingMode: SchedulingMode = SchedulingMode.withName( + System.getProperty("spark.scheduler.mode", "FIFO")) + + // This is a var so that we can reset it for testing purposes. + private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) + + override def setDAGScheduler(dagScheduler: DAGScheduler) { + this.dagScheduler = dagScheduler + } + + def initialize(context: SchedulerBackend) { + backend = context + // temporarily set rootPool name to empty + rootPool = new Pool("", schedulingMode, 0, 0) + schedulableBuilder = { + schedulingMode match { + case SchedulingMode.FIFO => + new FIFOSchedulableBuilder(rootPool) + case SchedulingMode.FAIR => + new FairSchedulableBuilder(rootPool) + } + } + schedulableBuilder.buildPools() + } + + def newTaskId(): Long = nextTaskId.getAndIncrement() + + override def start() { + backend.start() + + if (!isLocal && System.getProperty("spark.speculation", "false").toBoolean) { + new Thread("TaskScheduler speculation check") { + setDaemon(true) + + override def run() { + logInfo("Starting speculative execution thread") + while (true) { + try { + Thread.sleep(SPECULATION_INTERVAL) + } catch { + case e: InterruptedException => {} + } + checkSpeculatableTasks() + } + } + }.start() + } + } + + override def submitTasks(taskSet: TaskSet) { + val tasks = taskSet.tasks + logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks") + this.synchronized { + val manager = new TaskSetManager(this, taskSet, MAX_TASK_FAILURES) + activeTaskSets(taskSet.id) = manager + schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) + taskSetTaskIds(taskSet.id) = new HashSet[Long]() + + if (!isLocal && !hasReceivedTask) { + starvationTimer.scheduleAtFixedRate(new TimerTask() { + override def run() { + if (!hasLaunchedTask) { + logWarning("Initial job has not accepted any resources; " + + "check your cluster UI to ensure that workers are registered " + + "and have sufficient memory") + } else { + this.cancel() + } + } + }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) + } + hasReceivedTask = true + } + backend.reviveOffers() + } + + override def cancelTasks(stageId: Int): Unit = synchronized { + logInfo("Cancelling stage " + stageId) + activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) => + // There are two possible cases here: + // 1. The task set manager has been created and some tasks have been scheduled. + // In this case, send a kill signal to the executors to kill the task and then abort + // the stage. + // 2. The task set manager has been created but no tasks has been scheduled. In this case, + // simply abort the stage. + val taskIds = taskSetTaskIds(tsm.taskSet.id) + if (taskIds.size > 0) { + taskIds.foreach { tid => + val execId = taskIdToExecutorId(tid) + backend.killTask(tid, execId) + } + } + tsm.error("Stage %d was cancelled".format(stageId)) + } + } + + def taskSetFinished(manager: TaskSetManager): Unit = synchronized { + // Check to see if the given task set has been removed. This is possible in the case of + // multiple unrecoverable task failures (e.g. if the entire task set is killed when it has + // more than one running tasks). + if (activeTaskSets.contains(manager.taskSet.id)) { + activeTaskSets -= manager.taskSet.id + manager.parent.removeSchedulable(manager) + logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) + taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) + taskIdToExecutorId --= taskSetTaskIds(manager.taskSet.id) + taskSetTaskIds.remove(manager.taskSet.id) + } + } + + /** + * Called by cluster manager to offer resources on slaves. We respond by asking our active task + * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so + * that tasks are balanced across the cluster. + */ + def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = synchronized { + 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) + } + } + + // 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 sortedTaskSets = rootPool.getSortedTaskSetQueue() + for (taskSet <- sortedTaskSets) { + logDebug("parentName: %s, name: %s, runningTasks: %s".format( + 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 (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 <- taskSet.resourceOffer(execId, host, availableCpus(i), maxLocality)) { + tasks(i) += task + val tid = task.taskId + taskIdToTaskSetId(tid) = taskSet.taskSet.id + taskSetTaskIds(taskSet.taskSet.id) += tid + taskIdToExecutorId(tid) = execId + activeExecutorIds += execId + executorsByHost(host) += execId + availableCpus(i) -= 1 + launchedTask = true + } + } + } while (launchedTask) + } + + if (tasks.size > 0) { + hasLaunchedTask = true + } + return tasks + } + + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + var failedExecutor: Option[String] = None + var taskFailed = false + synchronized { + try { + if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) { + // We lost this entire executor, so remember that it's gone + val execId = taskIdToExecutorId(tid) + if (activeExecutorIds.contains(execId)) { + removeExecutor(execId) + failedExecutor = Some(execId) + } + } + taskIdToTaskSetId.get(tid) match { + case Some(taskSetId) => + if (TaskState.isFinished(state)) { + taskIdToTaskSetId.remove(tid) + if (taskSetTaskIds.contains(taskSetId)) { + taskSetTaskIds(taskSetId) -= tid + } + taskIdToExecutorId.remove(tid) + } + if (state == TaskState.FAILED) { + taskFailed = true + } + activeTaskSets.get(taskSetId).foreach { taskSet => + if (state == TaskState.FINISHED) { + taskSet.removeRunningTask(tid) + taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) + } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { + taskSet.removeRunningTask(tid) + taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData) + } + } + case None => + logInfo("Ignoring update from TID " + tid + " because its task set is gone") + } + } catch { + case e: Exception => logError("Exception in statusUpdate", e) + } + } + // Update the DAGScheduler without holding a lock on this, since that can deadlock + if (failedExecutor != None) { + dagScheduler.executorLost(failedExecutor.get) + backend.reviveOffers() + } + if (taskFailed) { + // Also revive offers if a task had failed for some reason other than host lost + backend.reviveOffers() + } + } + + def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long) { + taskSetManager.handleTaskGettingResult(tid) + } + + def handleSuccessfulTask( + taskSetManager: TaskSetManager, + tid: Long, + taskResult: DirectTaskResult[_]) = synchronized { + taskSetManager.handleSuccessfulTask(tid, taskResult) + } + + def handleFailedTask( + taskSetManager: TaskSetManager, + tid: Long, + taskState: TaskState, + reason: Option[TaskEndReason]) = synchronized { + taskSetManager.handleFailedTask(tid, taskState, reason) + if (taskState == TaskState.FINISHED) { + // The task finished successfully but the result was lost, so we should revive offers. + backend.reviveOffers() + } + } + + def error(message: String) { + synchronized { + if (activeTaskSets.size > 0) { + // Have each task set throw a SparkException with the error + for ((taskSetId, manager) <- activeTaskSets) { + try { + manager.error(message) + } catch { + case e: Exception => logError("Exception in error callback", e) + } + } + } else { + // No task sets are active but we still got an error. Just exit since this + // must mean the error is during registration. + // It might be good to do something smarter here in the future. + logError("Exiting due to error from task scheduler: " + message) + System.exit(1) + } + } + } + + override def stop() { + if (backend != null) { + backend.stop() + } + if (taskResultGetter != null) { + taskResultGetter.stop() + } + + // sleeping for an arbitrary 5 seconds : to ensure that messages are sent out. + // TODO: Do something better ! + Thread.sleep(5000L) + } + + override def defaultParallelism() = backend.defaultParallelism() + + // Check for speculatable tasks in all our active jobs. + def checkSpeculatableTasks() { + var shouldRevive = false + synchronized { + shouldRevive = rootPool.checkSpeculatableTasks() + } + if (shouldRevive) { + backend.reviveOffers() + } + } + + // Check for pending tasks in all our active jobs. + def hasPendingTasks: Boolean = { + synchronized { + rootPool.hasPendingTasks() + } + } + + def executorLost(executorId: String, reason: ExecutorLossReason) { + var failedExecutor: Option[String] = None + + synchronized { + if (activeExecutorIds.contains(executorId)) { + val hostPort = executorIdToHost(executorId) + logError("Lost executor %s on %s: %s".format(executorId, hostPort, reason)) + removeExecutor(executorId) + failedExecutor = Some(executorId) + } else { + // We may get multiple executorLost() calls with different loss reasons. For example, one + // may be triggered by a dropped connection from the slave while another may be a report + // of executor termination from Mesos. We produce log messages for both so we eventually + // report the termination reason. + logError("Lost an executor " + executorId + " (already removed): " + reason) + } + } + // Call dagScheduler.executorLost without holding the lock on this to prevent deadlock + if (failedExecutor != None) { + dagScheduler.executorLost(failedExecutor.get) + backend.reviveOffers() + } + } + + /** Remove an executor from all our data structures and mark it as lost */ + private def removeExecutor(executorId: String) { + activeExecutorIds -= executorId + val host = executorIdToHost(executorId) + val execs = executorsByHost.getOrElse(host, new HashSet) + execs -= executorId + if (execs.isEmpty) { + executorsByHost -= host + } + executorIdToHost -= executorId + rootPool.executorLost(executorId, host) + } + + def executorGained(execId: String, host: String) { + dagScheduler.executorGained(execId, host) + } + + def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized { + executorsByHost.get(host).map(_.toSet) + } + + def hasExecutorsAliveOnHost(host: String): Boolean = synchronized { + executorsByHost.contains(host) + } + + def isExecutorAlive(execId: String): Boolean = synchronized { + activeExecutorIds.contains(execId) + } + + // By default, rack is unknown + def getRackForHost(value: String): Option[String] = None +} + + +private[spark] object ClusterScheduler { + /** + * 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 + + // order keyList based on population of value in map + val keyList = _keyList.sortWith( + (left, right) => map(left).size > map(right).size + ) + + val retval = new ArrayBuffer[T](keyList.size * 2) + var index = 0 + var found = true + + while (found) { + found = false + for (key <- keyList) { + val containerList: ArrayBuffer[T] = map.get(key).getOrElse(null) + assert(containerList != null) + // Get the index'th entry for this host - if present + if (index < containerList.size){ + retval += containerList.apply(index) + found = true + } + } + index += 1 + } + + retval.toList + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 5408fa7353..a77ff35323 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.Utils /** * Runs a thread pool that deserializes and remotely fetches (if necessary) task results. */ -private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskScheduler) +private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler) extends Logging { private val THREADS = System.getProperty("spark.resultGetter.threads", "4").toInt private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala deleted file mode 100644 index b4ec695ece..0000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ /dev/null @@ -1,492 +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.spark.scheduler - -import java.nio.ByteBuffer -import java.util.concurrent.atomic.AtomicLong -import java.util.{TimerTask, Timer} - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet - -import org.apache.spark._ -import org.apache.spark.TaskState.TaskState -import org.apache.spark.scheduler.SchedulingMode.SchedulingMode - -/** - * Schedules tasks for a single SparkContext. Receives a set of tasks from the DAGScheduler for - * each stage, and is responsible for sending tasks to executors, running them, retrying if there - * are failures, and mitigating stragglers. Returns events to the DAGScheduler. - * - * Clients should first call 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 can also work with a local setup by using a LocalBackend and setting isLocal to true. - * 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 TaskScheduler(val sc: SparkContext, isLocal: Boolean = false) extends 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 - - // TaskSetManagers are not thread safe, so any access to one should be synchronized - // on this class. - val activeTaskSets = new HashMap[String, TaskSetManager] - - val taskIdToTaskSetId = new HashMap[Long, String] - val taskIdToExecutorId = new HashMap[Long, String] - val taskSetTaskIds = new HashMap[String, HashSet[Long]] - - @volatile private var hasReceivedTask = false - @volatile private var hasLaunchedTask = false - private val starvationTimer = new Timer(true) - - // Incrementing task IDs - val nextTaskId = new AtomicLong(0) - - // Which executor IDs we have executors on - val activeExecutorIds = new 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 executorsByHost = new HashMap[String, HashSet[String]] - - private val executorIdToHost = new HashMap[String, String] - - // Listener object to pass upcalls into - var dagScheduler: DAGScheduler = null - - var backend: SchedulerBackend = null - - val mapOutputTracker = SparkEnv.get.mapOutputTracker - - var schedulableBuilder: SchedulableBuilder = null - var rootPool: Pool = null - // default scheduler is FIFO - val schedulingMode: SchedulingMode = SchedulingMode.withName( - System.getProperty("spark.scheduler.mode", "FIFO")) - - // This is a var so that we can reset it for testing purposes. - private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) - - def setDAGScheduler(dagScheduler: DAGScheduler) { - this.dagScheduler = dagScheduler - } - - def initialize(context: SchedulerBackend) { - backend = context - // temporarily set rootPool name to empty - rootPool = new Pool("", schedulingMode, 0, 0) - schedulableBuilder = { - schedulingMode match { - case SchedulingMode.FIFO => - new FIFOSchedulableBuilder(rootPool) - case SchedulingMode.FAIR => - new FairSchedulableBuilder(rootPool) - } - } - schedulableBuilder.buildPools() - } - - def newTaskId(): Long = nextTaskId.getAndIncrement() - - def start() { - backend.start() - - if (!isLocal && System.getProperty("spark.speculation", "false").toBoolean) { - new Thread("TaskScheduler speculation check") { - setDaemon(true) - - override def run() { - logInfo("Starting speculative execution thread") - while (true) { - try { - Thread.sleep(SPECULATION_INTERVAL) - } catch { - case e: InterruptedException => {} - } - checkSpeculatableTasks() - } - } - }.start() - } - } - - def submitTasks(taskSet: TaskSet) { - val tasks = taskSet.tasks - logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks") - this.synchronized { - val manager = new TaskSetManager(this, taskSet) - activeTaskSets(taskSet.id) = manager - schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) - taskSetTaskIds(taskSet.id) = new HashSet[Long]() - - if (!isLocal && !hasReceivedTask) { - starvationTimer.scheduleAtFixedRate(new TimerTask() { - override def run() { - if (!hasLaunchedTask) { - logWarning("Initial job has not accepted any resources; " + - "check your cluster UI to ensure that workers are registered " + - "and have sufficient memory") - } else { - this.cancel() - } - } - }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) - } - hasReceivedTask = true - } - backend.reviveOffers() - } - - def cancelTasks(stageId: Int): Unit = synchronized { - logInfo("Cancelling stage " + stageId) - activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) => - // There are two possible cases here: - // 1. The task set manager has been created and some tasks have been scheduled. - // In this case, send a kill signal to the executors to kill the task and then abort - // the stage. - // 2. The task set manager has been created but no tasks has been scheduled. In this case, - // simply abort the stage. - val taskIds = taskSetTaskIds(tsm.taskSet.id) - if (taskIds.size > 0) { - taskIds.foreach { tid => - val execId = taskIdToExecutorId(tid) - backend.killTask(tid, execId) - } - } - tsm.error("Stage %d was cancelled".format(stageId)) - } - } - - def taskSetFinished(manager: TaskSetManager): Unit = synchronized { - // Check to see if the given task set has been removed. This is possible in the case of - // multiple unrecoverable task failures (e.g. if the entire task set is killed when it has - // more than one running tasks). - if (activeTaskSets.contains(manager.taskSet.id)) { - activeTaskSets -= manager.taskSet.id - manager.parent.removeSchedulable(manager) - logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) - taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) - taskIdToExecutorId --= taskSetTaskIds(manager.taskSet.id) - taskSetTaskIds.remove(manager.taskSet.id) - } - } - - /** - * Called by cluster manager to offer resources on slaves. We respond by asking our active task - * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so - * that tasks are balanced across the cluster. - */ - def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = synchronized { - 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) - } - } - - // 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 sortedTaskSets = rootPool.getSortedTaskSetQueue() - for (taskSet <- sortedTaskSets) { - logDebug("parentName: %s, name: %s, runningTasks: %s".format( - 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 (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 <- taskSet.resourceOffer(execId, host, availableCpus(i), maxLocality)) { - tasks(i) += task - val tid = task.taskId - taskIdToTaskSetId(tid) = taskSet.taskSet.id - taskSetTaskIds(taskSet.taskSet.id) += tid - taskIdToExecutorId(tid) = execId - activeExecutorIds += execId - executorsByHost(host) += execId - availableCpus(i) -= 1 - launchedTask = true - } - } - } while (launchedTask) - } - - if (tasks.size > 0) { - hasLaunchedTask = true - } - return tasks - } - - def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { - var failedExecutor: Option[String] = None - var taskFailed = false - synchronized { - try { - if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) { - // We lost this entire executor, so remember that it's gone - val execId = taskIdToExecutorId(tid) - if (activeExecutorIds.contains(execId)) { - removeExecutor(execId) - failedExecutor = Some(execId) - } - } - taskIdToTaskSetId.get(tid) match { - case Some(taskSetId) => - if (TaskState.isFinished(state)) { - taskIdToTaskSetId.remove(tid) - if (taskSetTaskIds.contains(taskSetId)) { - taskSetTaskIds(taskSetId) -= tid - } - taskIdToExecutorId.remove(tid) - } - if (state == TaskState.FAILED) { - taskFailed = true - } - activeTaskSets.get(taskSetId).foreach { taskSet => - if (state == TaskState.FINISHED) { - taskSet.removeRunningTask(tid) - taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) - } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { - taskSet.removeRunningTask(tid) - taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData) - } - } - case None => - logInfo("Ignoring update from TID " + tid + " because its task set is gone") - } - } catch { - case e: Exception => logError("Exception in statusUpdate", e) - } - } - // Update the DAGScheduler without holding a lock on this, since that can deadlock - if (failedExecutor != None) { - dagScheduler.executorLost(failedExecutor.get) - backend.reviveOffers() - } - if (taskFailed) { - // Also revive offers if a task had failed for some reason other than host lost - backend.reviveOffers() - } - } - - def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long) { - taskSetManager.handleTaskGettingResult(tid) - } - - def handleSuccessfulTask( - taskSetManager: TaskSetManager, - tid: Long, - taskResult: DirectTaskResult[_]) = synchronized { - taskSetManager.handleSuccessfulTask(tid, taskResult) - } - - def handleFailedTask( - taskSetManager: TaskSetManager, - tid: Long, - taskState: TaskState, - reason: Option[TaskEndReason]) = synchronized { - taskSetManager.handleFailedTask(tid, taskState, reason) - if (taskState == TaskState.FINISHED) { - // The task finished successfully but the result was lost, so we should revive offers. - backend.reviveOffers() - } - } - - def error(message: String) { - synchronized { - if (activeTaskSets.size > 0) { - // Have each task set throw a SparkException with the error - for ((taskSetId, manager) <- activeTaskSets) { - try { - manager.error(message) - } catch { - case e: Exception => logError("Exception in error callback", e) - } - } - } else { - // No task sets are active but we still got an error. Just exit since this - // must mean the error is during registration. - // It might be good to do something smarter here in the future. - logError("Exiting due to error from task scheduler: " + message) - System.exit(1) - } - } - } - - def stop() { - if (backend != null) { - backend.stop() - } - if (taskResultGetter != null) { - taskResultGetter.stop() - } - - // sleeping for an arbitrary 5 seconds : to ensure that messages are sent out. - // TODO: Do something better ! - Thread.sleep(5000L) - } - - def defaultParallelism() = backend.defaultParallelism() - - // Check for speculatable tasks in all our active jobs. - def checkSpeculatableTasks() { - var shouldRevive = false - synchronized { - shouldRevive = rootPool.checkSpeculatableTasks() - } - if (shouldRevive) { - backend.reviveOffers() - } - } - - // Check for pending tasks in all our active jobs. - def hasPendingTasks: Boolean = { - synchronized { - rootPool.hasPendingTasks() - } - } - - def executorLost(executorId: String, reason: ExecutorLossReason) { - var failedExecutor: Option[String] = None - - synchronized { - if (activeExecutorIds.contains(executorId)) { - val hostPort = executorIdToHost(executorId) - logError("Lost executor %s on %s: %s".format(executorId, hostPort, reason)) - removeExecutor(executorId) - failedExecutor = Some(executorId) - } else { - // We may get multiple executorLost() calls with different loss reasons. For example, one - // may be triggered by a dropped connection from the slave while another may be a report - // of executor termination from Mesos. We produce log messages for both so we eventually - // report the termination reason. - logError("Lost an executor " + executorId + " (already removed): " + reason) - } - } - // Call dagScheduler.executorLost without holding the lock on this to prevent deadlock - if (failedExecutor != None) { - dagScheduler.executorLost(failedExecutor.get) - backend.reviveOffers() - } - } - - /** Remove an executor from all our data structures and mark it as lost */ - private def removeExecutor(executorId: String) { - activeExecutorIds -= executorId - val host = executorIdToHost(executorId) - val execs = executorsByHost.getOrElse(host, new HashSet) - execs -= executorId - if (execs.isEmpty) { - executorsByHost -= host - } - executorIdToHost -= executorId - rootPool.executorLost(executorId, host) - } - - def executorGained(execId: String, host: String) { - dagScheduler.executorGained(execId, host) - } - - def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized { - executorsByHost.get(host).map(_.toSet) - } - - def hasExecutorsAliveOnHost(host: String): Boolean = synchronized { - executorsByHost.contains(host) - } - - def isExecutorAlive(execId: String): Boolean = synchronized { - activeExecutorIds.contains(execId) - } - - // By default, rack is unknown - def getRackForHost(value: String): Option[String] = None - - /** - * Invoked after the system has successfully been initialized. YARN uses this to bootstrap - * allocation of resources based on preferred locations, wait for slave registrations, etc. - */ - def postStartHook() { } -} - - -private[spark] object TaskScheduler { - /** - * 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 - - // order keyList based on population of value in map - val keyList = _keyList.sortWith( - (left, right) => map(left).size > map(right).size - ) - - val retval = new ArrayBuffer[T](keyList.size * 2) - var index = 0 - var found = true - - while (found) { - found = false - for (key <- keyList) { - val containerList: ArrayBuffer[T] = map.get(key).getOrElse(null) - assert(containerList != null) - // Get the index'th entry for this host - if present - if (index < containerList.size){ - retval += containerList.apply(index) - found = true - } - } - index += 1 - } - - retval.toList - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 90b6519027..8757d7fd2a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -40,19 +40,22 @@ import org.apache.spark.util.{SystemClock, Clock} * * 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. + * + * @param sched the ClusterScheduler associated with the TaskSetManager + * @param taskSet the TaskSet to manage scheduling for + * @param maxTaskFailures if any particular task fails more than this number of times, the entire + * task set will be aborted */ private[spark] class TaskSetManager( - sched: TaskScheduler, + sched: ClusterScheduler, val taskSet: TaskSet, + val maxTaskFailures: Int, clock: Clock = SystemClock) extends Schedulable with Logging { // CPUs to request per task 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 - // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble @@ -521,10 +524,10 @@ private[spark] class TaskSetManager( addPendingTask(index) if (state != TaskState.KILLED) { numFailures(index) += 1 - if (numFailures(index) > MAX_TASK_FAILURES) { + if (numFailures(index) > maxTaskFailures) { logError("Task %s:%d failed more than %d times; aborting job".format( - taskSet.id, index, MAX_TASK_FAILURES)) - abort("Task %s:%d failed more than %d times".format(taskSet.id, index, MAX_TASK_FAILURES)) + taskSet.id, index, maxTaskFailures)) + abort("Task %s:%d failed more than %d times".format(taskSet.id, index, maxTaskFailures)) } } } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index b8ac498527..f5548fc2da 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -29,7 +29,7 @@ import akka.util.Duration import akka.util.duration._ import org.apache.spark.{SparkException, Logging, TaskState} -import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, TaskScheduler, +import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, ClusterScheduler, WorkerOffer} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.Utils @@ -43,7 +43,7 @@ import org.apache.spark.util.Utils * (spark.deploy.*). */ private[spark] -class CoarseGrainedSchedulerBackend(scheduler: TaskScheduler, actorSystem: ActorSystem) +class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: ActorSystem) extends SchedulerBackend with Logging { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index a589e7456f..40fdfcddb1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -21,10 +21,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.scheduler.TaskScheduler +import org.apache.spark.scheduler.ClusterScheduler private[spark] class SimrSchedulerBackend( - scheduler: TaskScheduler, + scheduler: ClusterScheduler, sc: SparkContext, driverFilePath: String) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) 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 15c600a1ec..acf15dbc40 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 @@ -22,11 +22,11 @@ import scala.collection.mutable.HashMap import org.apache.spark.{Logging, SparkContext} import org.apache.spark.deploy.client.{Client, ClientListener} import org.apache.spark.deploy.{Command, ApplicationDescription} -import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskScheduler} +import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, ClusterScheduler} import org.apache.spark.util.Utils private[spark] class SparkDeploySchedulerBackend( - scheduler: TaskScheduler, + scheduler: ClusterScheduler, sc: SparkContext, masters: Array[String], appName: String) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 310da0027e..226ea46cc7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -30,7 +30,7 @@ import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} import org.apache.spark.{SparkException, Logging, SparkContext, TaskState} -import org.apache.spark.scheduler.TaskScheduler +import org.apache.spark.scheduler.ClusterScheduler import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend /** @@ -44,7 +44,7 @@ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend * remove this. */ private[spark] class CoarseMesosSchedulerBackend( - scheduler: TaskScheduler, + scheduler: ClusterScheduler, sc: SparkContext, master: String, appName: String) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index c0e99df0b6..3acad1bb46 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -31,7 +31,7 @@ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTas import org.apache.spark.{Logging, SparkException, SparkContext, TaskState} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SchedulerBackend, SlaveLost, - TaskDescription, TaskScheduler, WorkerOffer} + TaskDescription, ClusterScheduler, WorkerOffer} import org.apache.spark.util.Utils /** @@ -40,7 +40,7 @@ import org.apache.spark.util.Utils * from multiple apps can run on different cores) and in time (a core can switch ownership). */ private[spark] class MesosSchedulerBackend( - scheduler: TaskScheduler, + scheduler: ClusterScheduler, sc: SparkContext, master: String, appName: String) diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 96c3a03602..3e9d31cd5e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -24,16 +24,17 @@ import akka.actor.{Actor, ActorRef, Props} import org.apache.spark.{SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} -import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, WorkerOffer} +import org.apache.spark.scheduler.{SchedulerBackend, ClusterScheduler, WorkerOffer} /** - * LocalBackend sits behind a TaskScheduler and handles launching tasks on a single Executor - * (created by the LocalBackend) running locally. + * LocalBackend is used when running a local version of Spark where the executor, backend, and + * master all run in the same JVM. It sits behind a ClusterScheduler and handles launching tasks + * on a single Executor (created by the LocalBackend) running locally. * * THREADING: Because methods can be called both from the Executor and the TaskScheduler, and * because the Executor class is not thread safe, all methods are synchronized. */ -private[spark] class LocalBackend(scheduler: TaskScheduler, private val totalCores: Int) +private[spark] class LocalBackend(scheduler: ClusterScheduler, private val totalCores: Int) extends SchedulerBackend with ExecutorBackend { private var freeCores = totalCores diff --git a/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala new file mode 100644 index 0000000000..96adcf7198 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala @@ -0,0 +1,265 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 org.scalatest.FunSuite +import org.scalatest.BeforeAndAfter + +import org.apache.spark._ +import scala.collection.mutable.ArrayBuffer + +import java.util.Properties + +class FakeTaskSetManager( + initPriority: Int, + initStageId: Int, + initNumTasks: Int, + taskScheduler: ClusterScheduler, + taskSet: TaskSet) + extends TaskSetManager(taskScheduler, taskSet, 1) { + + parent = null + weight = 1 + minShare = 2 + runningTasks = 0 + priority = initPriority + stageId = initStageId + name = "TaskSet_"+stageId + override val numTasks = initNumTasks + tasksSuccessful = 0 + + def increaseRunningTasks(taskNum: Int) { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } + } + + def decreaseRunningTasks(taskNum: Int) { + runningTasks -= taskNum + if (parent != null) { + parent.decreaseRunningTasks(taskNum) + } + } + + override def addSchedulable(schedulable: Schedulable) { + } + + override def removeSchedulable(schedulable: Schedulable) { + } + + override def getSchedulableByName(name: String): Schedulable = { + return null + } + + override def executorLost(executorId: String, host: String): Unit = { + } + + override def resourceOffer( + execId: String, + host: String, + availableCpus: Int, + maxLocality: TaskLocality.TaskLocality) + : Option[TaskDescription] = + { + if (tasksSuccessful + runningTasks < numTasks) { + increaseRunningTasks(1) + return Some(new TaskDescription(0, execId, "task 0:0", 0, null)) + } + return None + } + + override def checkSpeculatableTasks(): Boolean = { + return true + } + + def taskFinished() { + decreaseRunningTasks(1) + tasksSuccessful +=1 + if (tasksSuccessful == numTasks) { + parent.removeSchedulable(this) + } + } + + def abort() { + decreaseRunningTasks(runningTasks) + parent.removeSchedulable(this) + } +} + +class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging { + + 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 = { + val taskSetQueue = rootPool.getSortedTaskSetQueue() + /* Just for Test*/ + for (manager <- taskSetQueue) { + logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format( + manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks)) + } + for (taskSet <- taskSetQueue) { + taskSet.resourceOffer("execId_1", "hostname_1", 1, TaskLocality.ANY) match { + case Some(task) => + return taskSet.stageId + case None => {} + } + } + -1 + } + + def checkTaskSetId(rootPool: Pool, expectedTaskSetId: Int) { + assert(resourceOffer(rootPool) === expectedTaskSetId) + } + + test("FIFO Scheduler Test") { + sc = new SparkContext("local", "TaskSchedulerSuite") + val taskScheduler = new ClusterScheduler(sc) + var tasks = ArrayBuffer[Task[_]]() + val task = new FakeTask(0) + tasks += task + val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + + val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0) + val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) + schedulableBuilder.buildPools() + + val taskSetManager0 = createDummyTaskSetManager(0, 0, 2, taskScheduler, taskSet) + val taskSetManager1 = createDummyTaskSetManager(0, 1, 2, taskScheduler, taskSet) + val taskSetManager2 = createDummyTaskSetManager(0, 2, 2, taskScheduler, taskSet) + schedulableBuilder.addTaskSetManager(taskSetManager0, null) + schedulableBuilder.addTaskSetManager(taskSetManager1, null) + schedulableBuilder.addTaskSetManager(taskSetManager2, null) + + checkTaskSetId(rootPool, 0) + resourceOffer(rootPool) + checkTaskSetId(rootPool, 1) + resourceOffer(rootPool) + taskSetManager1.abort() + checkTaskSetId(rootPool, 2) + } + + test("Fair Scheduler Test") { + sc = new SparkContext("local", "TaskSchedulerSuite") + val taskScheduler = new ClusterScheduler(sc) + var tasks = ArrayBuffer[Task[_]]() + val task = new FakeTask(0) + tasks += task + val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + + val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() + System.setProperty("spark.scheduler.allocation.file", xmlPath) + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val schedulableBuilder = new FairSchedulableBuilder(rootPool) + schedulableBuilder.buildPools() + + assert(rootPool.getSchedulableByName("default") != null) + assert(rootPool.getSchedulableByName("1") != null) + assert(rootPool.getSchedulableByName("2") != null) + assert(rootPool.getSchedulableByName("3") != null) + assert(rootPool.getSchedulableByName("1").minShare === 2) + assert(rootPool.getSchedulableByName("1").weight === 1) + assert(rootPool.getSchedulableByName("2").minShare === 3) + assert(rootPool.getSchedulableByName("2").weight === 1) + assert(rootPool.getSchedulableByName("3").minShare === 0) + assert(rootPool.getSchedulableByName("3").weight === 1) + + val properties1 = new Properties() + properties1.setProperty("spark.scheduler.pool","1") + val properties2 = new Properties() + properties2.setProperty("spark.scheduler.pool","2") + + val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, taskScheduler, taskSet) + val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, taskScheduler, taskSet) + val taskSetManager12 = createDummyTaskSetManager(1, 2, 2, taskScheduler, taskSet) + schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) + schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) + schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) + + val taskSetManager23 = createDummyTaskSetManager(2, 3, 2, taskScheduler, taskSet) + val taskSetManager24 = createDummyTaskSetManager(2, 4, 2, taskScheduler, taskSet) + schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) + schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) + + checkTaskSetId(rootPool, 0) + checkTaskSetId(rootPool, 3) + checkTaskSetId(rootPool, 3) + checkTaskSetId(rootPool, 1) + checkTaskSetId(rootPool, 4) + checkTaskSetId(rootPool, 2) + checkTaskSetId(rootPool, 2) + checkTaskSetId(rootPool, 4) + + taskSetManager12.taskFinished() + assert(rootPool.getSchedulableByName("1").runningTasks === 3) + taskSetManager24.abort() + assert(rootPool.getSchedulableByName("2").runningTasks === 2) + } + + test("Nested Pool Test") { + sc = new SparkContext("local", "TaskSchedulerSuite") + val taskScheduler = new ClusterScheduler(sc) + var tasks = ArrayBuffer[Task[_]]() + val task = new FakeTask(0) + tasks += task + val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1) + val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1) + rootPool.addSchedulable(pool0) + rootPool.addSchedulable(pool1) + + val pool00 = new Pool("00", SchedulingMode.FAIR, 2, 2) + val pool01 = new Pool("01", SchedulingMode.FAIR, 1, 1) + pool0.addSchedulable(pool00) + pool0.addSchedulable(pool01) + + val pool10 = new Pool("10", SchedulingMode.FAIR, 2, 2) + val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1) + pool1.addSchedulable(pool10) + pool1.addSchedulable(pool11) + + val taskSetManager000 = createDummyTaskSetManager(0, 0, 5, taskScheduler, taskSet) + val taskSetManager001 = createDummyTaskSetManager(0, 1, 5, taskScheduler, taskSet) + pool00.addSchedulable(taskSetManager000) + pool00.addSchedulable(taskSetManager001) + + val taskSetManager010 = createDummyTaskSetManager(1, 2, 5, taskScheduler, taskSet) + val taskSetManager011 = createDummyTaskSetManager(1, 3, 5, taskScheduler, taskSet) + pool01.addSchedulable(taskSetManager010) + pool01.addSchedulable(taskSetManager011) + + val taskSetManager100 = createDummyTaskSetManager(2, 4, 5, taskScheduler, taskSet) + val taskSetManager101 = createDummyTaskSetManager(2, 5, 5, taskScheduler, taskSet) + pool10.addSchedulable(taskSetManager100) + pool10.addSchedulable(taskSetManager101) + + val taskSetManager110 = createDummyTaskSetManager(3, 6, 5, taskScheduler, taskSet) + val taskSetManager111 = createDummyTaskSetManager(3, 7, 5, taskScheduler, taskSet) + pool11.addSchedulable(taskSetManager110) + pool11.addSchedulable(taskSetManager111) + + checkTaskSetId(rootPool, 0) + checkTaskSetId(rootPool, 4) + checkTaskSetId(rootPool, 6) + checkTaskSetId(rootPool, 2) + } +} 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 5b5a2178f3..24689a7093 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -37,7 +37,7 @@ import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} * TaskScheduler that records the task sets that the DAGScheduler requested executed. */ class TaskSetRecordingTaskScheduler(sc: SparkContext, - mapOutputTrackerMaster: MapOutputTrackerMaster) extends TaskScheduler(sc) { + mapOutputTrackerMaster: MapOutputTrackerMaster) extends ClusterScheduler(sc) { /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() override def start() = {} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 30e6bc5721..2ac2d7a36a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.storage.TaskResultBlockId * Used to test the case where a BlockManager evicts the task result (or dies) before the * TaskResult is retrieved. */ -class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskScheduler) +class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler) extends TaskResultGetter(sparkEnv, scheduler) { var removedResult = false @@ -91,8 +91,8 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA test("task retried if result missing from block manager") { // If this test hangs, it's probably because no resource offers were made after the task // failed. - val scheduler: TaskScheduler = sc.taskScheduler match { - case clusterScheduler: TaskScheduler => + val scheduler: ClusterScheduler = sc.taskScheduler match { + case clusterScheduler: ClusterScheduler => clusterScheduler case _ => assert(false, "Expect local cluster to use TaskScheduler") diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerSuite.scala deleted file mode 100644 index bfbffdf261..0000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerSuite.scala +++ /dev/null @@ -1,265 +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.spark.scheduler - -import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter - -import org.apache.spark._ -import scala.collection.mutable.ArrayBuffer - -import java.util.Properties - -class FakeTaskSetManager( - initPriority: Int, - initStageId: Int, - initNumTasks: Int, - taskScheduler: TaskScheduler, - taskSet: TaskSet) - extends TaskSetManager(taskScheduler, taskSet) { - - parent = null - weight = 1 - minShare = 2 - runningTasks = 0 - priority = initPriority - stageId = initStageId - name = "TaskSet_"+stageId - override val numTasks = initNumTasks - tasksSuccessful = 0 - - def increaseRunningTasks(taskNum: Int) { - runningTasks += taskNum - if (parent != null) { - parent.increaseRunningTasks(taskNum) - } - } - - def decreaseRunningTasks(taskNum: Int) { - runningTasks -= taskNum - if (parent != null) { - parent.decreaseRunningTasks(taskNum) - } - } - - override def addSchedulable(schedulable: Schedulable) { - } - - override def removeSchedulable(schedulable: Schedulable) { - } - - override def getSchedulableByName(name: String): Schedulable = { - return null - } - - override def executorLost(executorId: String, host: String): Unit = { - } - - override def resourceOffer( - execId: String, - host: String, - availableCpus: Int, - maxLocality: TaskLocality.TaskLocality) - : Option[TaskDescription] = - { - if (tasksSuccessful + runningTasks < numTasks) { - increaseRunningTasks(1) - return Some(new TaskDescription(0, execId, "task 0:0", 0, null)) - } - return None - } - - override def checkSpeculatableTasks(): Boolean = { - return true - } - - def taskFinished() { - decreaseRunningTasks(1) - tasksSuccessful +=1 - if (tasksSuccessful == numTasks) { - parent.removeSchedulable(this) - } - } - - def abort() { - decreaseRunningTasks(runningTasks) - parent.removeSchedulable(this) - } -} - -class TaskSchedulerSuite extends FunSuite with LocalSparkContext with Logging { - - def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: TaskScheduler, taskSet: TaskSet): FakeTaskSetManager = { - new FakeTaskSetManager(priority, stage, numTasks, cs , taskSet) - } - - def resourceOffer(rootPool: Pool): Int = { - val taskSetQueue = rootPool.getSortedTaskSetQueue() - /* Just for Test*/ - for (manager <- taskSetQueue) { - logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format( - manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks)) - } - for (taskSet <- taskSetQueue) { - taskSet.resourceOffer("execId_1", "hostname_1", 1, TaskLocality.ANY) match { - case Some(task) => - return taskSet.stageId - case None => {} - } - } - -1 - } - - def checkTaskSetId(rootPool: Pool, expectedTaskSetId: Int) { - assert(resourceOffer(rootPool) === expectedTaskSetId) - } - - test("FIFO Scheduler Test") { - sc = new SparkContext("local", "TaskSchedulerSuite") - val taskScheduler = new TaskScheduler(sc) - var tasks = ArrayBuffer[Task[_]]() - val task = new FakeTask(0) - tasks += task - val taskSet = new TaskSet(tasks.toArray,0,0,0,null) - - val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0) - val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) - schedulableBuilder.buildPools() - - val taskSetManager0 = createDummyTaskSetManager(0, 0, 2, taskScheduler, taskSet) - val taskSetManager1 = createDummyTaskSetManager(0, 1, 2, taskScheduler, taskSet) - val taskSetManager2 = createDummyTaskSetManager(0, 2, 2, taskScheduler, taskSet) - schedulableBuilder.addTaskSetManager(taskSetManager0, null) - schedulableBuilder.addTaskSetManager(taskSetManager1, null) - schedulableBuilder.addTaskSetManager(taskSetManager2, null) - - checkTaskSetId(rootPool, 0) - resourceOffer(rootPool) - checkTaskSetId(rootPool, 1) - resourceOffer(rootPool) - taskSetManager1.abort() - checkTaskSetId(rootPool, 2) - } - - test("Fair Scheduler Test") { - sc = new SparkContext("local", "TaskSchedulerSuite") - val taskScheduler = new TaskScheduler(sc) - var tasks = ArrayBuffer[Task[_]]() - val task = new FakeTask(0) - tasks += task - val taskSet = new TaskSet(tasks.toArray,0,0,0,null) - - val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) - val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) - val schedulableBuilder = new FairSchedulableBuilder(rootPool) - schedulableBuilder.buildPools() - - assert(rootPool.getSchedulableByName("default") != null) - assert(rootPool.getSchedulableByName("1") != null) - assert(rootPool.getSchedulableByName("2") != null) - assert(rootPool.getSchedulableByName("3") != null) - assert(rootPool.getSchedulableByName("1").minShare === 2) - assert(rootPool.getSchedulableByName("1").weight === 1) - assert(rootPool.getSchedulableByName("2").minShare === 3) - assert(rootPool.getSchedulableByName("2").weight === 1) - assert(rootPool.getSchedulableByName("3").minShare === 0) - assert(rootPool.getSchedulableByName("3").weight === 1) - - val properties1 = new Properties() - properties1.setProperty("spark.scheduler.pool","1") - val properties2 = new Properties() - properties2.setProperty("spark.scheduler.pool","2") - - val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, taskScheduler, taskSet) - val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, taskScheduler, taskSet) - val taskSetManager12 = createDummyTaskSetManager(1, 2, 2, taskScheduler, taskSet) - schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) - schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) - schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) - - val taskSetManager23 = createDummyTaskSetManager(2, 3, 2, taskScheduler, taskSet) - val taskSetManager24 = createDummyTaskSetManager(2, 4, 2, taskScheduler, taskSet) - schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) - schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) - - checkTaskSetId(rootPool, 0) - checkTaskSetId(rootPool, 3) - checkTaskSetId(rootPool, 3) - checkTaskSetId(rootPool, 1) - checkTaskSetId(rootPool, 4) - checkTaskSetId(rootPool, 2) - checkTaskSetId(rootPool, 2) - checkTaskSetId(rootPool, 4) - - taskSetManager12.taskFinished() - assert(rootPool.getSchedulableByName("1").runningTasks === 3) - taskSetManager24.abort() - assert(rootPool.getSchedulableByName("2").runningTasks === 2) - } - - test("Nested Pool Test") { - sc = new SparkContext("local", "TaskSchedulerSuite") - val taskScheduler = new TaskScheduler(sc) - var tasks = ArrayBuffer[Task[_]]() - val task = new FakeTask(0) - tasks += task - val taskSet = new TaskSet(tasks.toArray,0,0,0,null) - - val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) - val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1) - val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1) - rootPool.addSchedulable(pool0) - rootPool.addSchedulable(pool1) - - val pool00 = new Pool("00", SchedulingMode.FAIR, 2, 2) - val pool01 = new Pool("01", SchedulingMode.FAIR, 1, 1) - pool0.addSchedulable(pool00) - pool0.addSchedulable(pool01) - - val pool10 = new Pool("10", SchedulingMode.FAIR, 2, 2) - val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1) - pool1.addSchedulable(pool10) - pool1.addSchedulable(pool11) - - val taskSetManager000 = createDummyTaskSetManager(0, 0, 5, taskScheduler, taskSet) - val taskSetManager001 = createDummyTaskSetManager(0, 1, 5, taskScheduler, taskSet) - pool00.addSchedulable(taskSetManager000) - pool00.addSchedulable(taskSetManager001) - - val taskSetManager010 = createDummyTaskSetManager(1, 2, 5, taskScheduler, taskSet) - val taskSetManager011 = createDummyTaskSetManager(1, 3, 5, taskScheduler, taskSet) - pool01.addSchedulable(taskSetManager010) - pool01.addSchedulable(taskSetManager011) - - val taskSetManager100 = createDummyTaskSetManager(2, 4, 5, taskScheduler, taskSet) - val taskSetManager101 = createDummyTaskSetManager(2, 5, 5, taskScheduler, taskSet) - pool10.addSchedulable(taskSetManager100) - pool10.addSchedulable(taskSetManager101) - - val taskSetManager110 = createDummyTaskSetManager(3, 6, 5, taskScheduler, taskSet) - val taskSetManager111 = createDummyTaskSetManager(3, 7, 5, taskScheduler, taskSet) - pool11.addSchedulable(taskSetManager110) - pool11.addSchedulable(taskSetManager111) - - checkTaskSetId(rootPool, 0) - checkTaskSetId(rootPool, 4) - checkTaskSetId(rootPool, 6) - checkTaskSetId(rootPool, 2) - } -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index fe3ea7b594..592bb11364 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -58,7 +58,7 @@ class FakeDAGScheduler(taskScheduler: FakeTaskScheduler) extends DAGScheduler(ta * to work, and these are required for locality in TaskSetManager. */ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* execId, host */) - extends TaskScheduler(sc) + extends ClusterScheduler(sc) { val startedTasks = new ArrayBuffer[Long] val endedTasks = new mutable.HashMap[Long, TaskEndReason] @@ -82,12 +82,13 @@ class TaskSetManagerSuite 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 + val MAX_TASK_FAILURES = 4 test("TaskSet with no preferences") { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = createTaskSet(1) - val manager = new TaskSetManager(sched, taskSet) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) // Offer a host with no CPUs assert(manager.resourceOffer("exec1", "host1", 0, ANY) === None) @@ -113,7 +114,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = createTaskSet(3) - val manager = new TaskSetManager(sched, taskSet) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) // First three offers should all find tasks for (i <- 0 until 3) { @@ -150,7 +151,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq() // Last task has no locality prefs ) val clock = new FakeClock - val manager = new TaskSetManager(sched, taskSet, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1, exec1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) @@ -196,7 +197,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host2")) ) val clock = new FakeClock - val manager = new TaskSetManager(sched, taskSet, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) @@ -233,7 +234,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host3")) ) val clock = new FakeClock - val manager = new TaskSetManager(sched, taskSet, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) @@ -261,7 +262,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = createTaskSet(1) val clock = new FakeClock - val manager = new TaskSetManager(sched, taskSet, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) @@ -278,17 +279,17 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = createTaskSet(1) val clock = new FakeClock - val manager = new TaskSetManager(sched, taskSet, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted // after the last failure. - (0 until manager.MAX_TASK_FAILURES).foreach { index => + (0 until MAX_TASK_FAILURES).foreach { index => val offerResult = manager.resourceOffer("exec1", "host1", 1, ANY) assert(offerResult != None, "Expect resource offer on iteration %s to return a task".format(index)) assert(offerResult.get.index === 0) manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, Some(TaskResultLost)) - if (index < manager.MAX_TASK_FAILURES) { + if (index < MAX_TASK_FAILURES) { assert(!sched.taskSetsFailed.contains(taskSet.id)) } else { assert(sched.taskSetsFailed.contains(taskSet.id)) -- cgit v1.2.3 From 46f9c6b858cf9737b7d46b22b75bfc847244331b Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Wed, 13 Nov 2013 15:46:41 -0800 Subject: Fixed naming issues and added back ability to specify max task failures. --- .../main/scala/org/apache/spark/SparkContext.scala | 17 +++- .../apache/spark/scheduler/ClusterScheduler.scala | 19 ++--- .../apache/spark/scheduler/SchedulerBackend.scala | 2 +- .../org/apache/spark/scheduler/TaskScheduler.scala | 56 +++++++++++++ .../apache/spark/scheduler/TaskSetManager.scala | 2 +- .../cluster/CoarseGrainedSchedulerBackend.scala | 2 +- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- .../test/scala/org/apache/spark/FailureSuite.scala | 20 ++--- .../spark/scheduler/ClusterSchedulerSuite.scala | 48 +++++------ .../apache/spark/scheduler/DAGSchedulerSuite.scala | 97 +++++++++++----------- .../spark/scheduler/TaskResultGetterSuite.scala | 13 +-- .../spark/scheduler/TaskSetManagerSuite.scala | 20 ++--- .../scheduler/cluster/YarnClusterScheduler.scala | 6 +- 13 files changed, 177 insertions(+), 127 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 10db2fa7e7..06bea0c535 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -156,6 +156,8 @@ class SparkContext( private[spark] var taskScheduler: TaskScheduler = { // Regular expression used for local[N] master format val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r + // Regular expression for local[N, maxRetries], used in tests with failing tasks + val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r // Regular expression for simulating a Spark cluster of [N, cores, memory] locally val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r // Regular expression for connecting to Spark deploy clusters @@ -165,19 +167,28 @@ class SparkContext( // Regular expression for connection to Simr cluster val SIMR_REGEX = """simr://(.*)""".r + // When running locally, don't try to re-execute tasks on failure. + val MAX_LOCAL_TASK_FAILURES = 0 + master match { case "local" => - val scheduler = new ClusterScheduler(this, isLocal = true) + val scheduler = new ClusterScheduler(this, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalBackend(scheduler, 1) scheduler.initialize(backend) scheduler case LOCAL_N_REGEX(threads) => - val scheduler = new ClusterScheduler(this, isLocal = true) + val scheduler = new ClusterScheduler(this, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalBackend(scheduler, threads.toInt) scheduler.initialize(backend) scheduler + case LOCAL_N_FAILURES_REGEX(threads, maxFailures) => + val scheduler = new ClusterScheduler(this, maxFailures.toInt, isLocal = true) + val backend = new LocalBackend(scheduler, threads.toInt) + scheduler.initialize(backend) + scheduler + case SPARK_REGEX(sparkUrl) => val scheduler = new ClusterScheduler(this) val masterUrls = sparkUrl.split(",").map("spark://" + _) @@ -200,7 +211,7 @@ class SparkContext( memoryPerSlaveInt, SparkContext.executorMemoryRequested)) } - val scheduler = new ClusterScheduler(this, isLocal = true) + val scheduler = new ClusterScheduler(this) val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val masterUrls = localCluster.start() diff --git a/core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala index c5d7ca0481..37d554715d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala @@ -46,8 +46,10 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode * 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, isLocal: Boolean = false) - extends TaskScheduler with Logging { +private[spark] class ClusterScheduler( + val sc: SparkContext, + val maxTaskFailures : Int = System.getProperty("spark.task.maxFailures", "4").toInt, + isLocal: Boolean = false) extends TaskScheduler with Logging { // How often to check for speculative tasks val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong @@ -59,15 +61,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext, isLocal: Boolean = f // on this class. val activeTaskSets = new HashMap[String, TaskSetManager] - val MAX_TASK_FAILURES = { - if (isLocal) { - // No sense in retrying if all tasks run locally! - 0 - } else { - System.getProperty("spark.task.maxFailures", "4").toInt - } - } - val taskIdToTaskSetId = new HashMap[Long, String] val taskIdToExecutorId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] @@ -142,7 +135,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext, isLocal: Boolean = f val tasks = taskSet.tasks logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks") this.synchronized { - val manager = new TaskSetManager(this, taskSet, MAX_TASK_FAILURES) + val manager = new TaskSetManager(this, taskSet, maxTaskFailures) activeTaskSets(taskSet.id) = manager schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) taskSetTaskIds(taskSet.id) = new HashSet[Long]() @@ -345,7 +338,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext, isLocal: Boolean = f // No task sets are active but we still got an error. Just exit since this // must mean the error is during registration. // It might be good to do something smarter here in the future. - logError("Exiting due to error from task scheduler: " + message) + logError("Exiting due to error from cluster scheduler: " + message) System.exit(1) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 1f0839a0e1..89aa098664 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkContext /** * A backend interface for scheduling systems that allows plugging in different ones under - * TaskScheduler. We assume a Mesos-like model where the application gets resource offers as + * ClusterScheduler. We assume a Mesos-like model where the application gets resource offers as * machines become available and can launch tasks on them. */ private[spark] trait SchedulerBackend { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala new file mode 100644 index 0000000000..17b6d97e90 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 org.apache.spark.scheduler.SchedulingMode.SchedulingMode + +/** + * Low-level task scheduler interface, currently implemented exclusively by the ClusterScheduler. + * This interface allows plugging in different task schedulers. Each TaskScheduler schedulers tasks + * for a single SparkContext. These schedulers get sets of tasks submitted to them from the + * DAGScheduler for each stage, and are responsible for sending the tasks to the cluster, running + * them, retrying if there are failures, and mitigating stragglers. They return events to the + * DAGScheduler. + */ +private[spark] trait TaskScheduler { + + def rootPool: Pool + + def schedulingMode: SchedulingMode + + def start(): Unit + + // Invoked after system has successfully initialized (typically in spark context). + // Yarn uses this to bootstrap allocation of resources based on preferred locations, wait for slave registerations, etc. + def postStartHook() { } + + // Disconnect from the cluster. + def stop(): Unit + + // Submit a sequence of tasks to run. + def submitTasks(taskSet: TaskSet): Unit + + // Cancel a stage. + def cancelTasks(stageId: Int) + + // Set the DAG scheduler for upcalls. This is guaranteed to be set before submitTasks is called. + def setDAGScheduler(dagScheduler: DAGScheduler): Unit + + // Get the default level of parallelism to use in the cluster, as a hint for sizing jobs. + def defaultParallelism(): Int +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 8757d7fd2a..bc35e53220 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.{SystemClock, Clock} /** - * Schedules the tasks within a single TaskSet in the TaskScheduler. This class keeps track of + * Schedules the tasks within a single TaskSet in the ClusterScheduler. This class keeps track 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, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 3bb715e7d0..3af02b42b2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -29,7 +29,7 @@ import akka.util.Duration import akka.util.duration._ import org.apache.spark.{SparkException, Logging, TaskState} -import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, ClusterScheduler, +import org.apache.spark.scheduler.{ClusterScheduler, SchedulerBackend, SlaveLost, TaskDescription, WorkerOffer} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 3acad1bb46..773b980c53 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -209,7 +209,7 @@ private[spark] class MesosSchedulerBackend( getResource(offer.getResourcesList, "cpus").toInt) } - // Call into the TaskScheduler + // Call into the ClusterScheduler val taskLists = scheduler.resourceOffers(offerableWorkers) // Build a list of Mesos tasks for each slave diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 2f7d6dff38..af448fcb37 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.FunSuite import SparkContext._ import org.apache.spark.util.NonSerializable @@ -37,20 +37,12 @@ object FailureSuiteState { } } -class FailureSuite extends FunSuite with LocalSparkContext with BeforeAndAfterAll { - - override def beforeAll { - System.setProperty("spark.task.maxFailures", "1") - } - - override def afterAll { - System.clearProperty("spark.task.maxFailures") - } +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") { - sc = new SparkContext("local[1]", "test") + sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3, 3).map { x => FailureSuiteState.synchronized { FailureSuiteState.tasksRun += 1 @@ -70,7 +62,7 @@ class FailureSuite extends FunSuite with LocalSparkContext with BeforeAndAfterAl // Run a map-reduce job in which a reduce task deterministically fails once. test("failure in a two-stage job") { - sc = new SparkContext("local[1]", "test") + sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => (x, x)).groupByKey(3).map { case (k, v) => FailureSuiteState.synchronized { @@ -90,7 +82,7 @@ class FailureSuite extends FunSuite with LocalSparkContext with BeforeAndAfterAl } test("failure because task results are not serializable") { - sc = new SparkContext("local[1]", "test") + sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => new NonSerializable) val thrown = intercept[SparkException] { @@ -103,7 +95,7 @@ class FailureSuite extends FunSuite with LocalSparkContext with BeforeAndAfterAl } test("failure because task closure is not serializable") { - sc = new SparkContext("local[1]", "test") + sc = new SparkContext("local[1,1]", "test") val a = new NonSerializable // Non-serializable closure in the final result stage diff --git a/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala index 96adcf7198..35a06c4875 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala @@ -29,9 +29,9 @@ class FakeTaskSetManager( initPriority: Int, initStageId: Int, initNumTasks: Int, - taskScheduler: ClusterScheduler, + clusterScheduler: ClusterScheduler, taskSet: TaskSet) - extends TaskSetManager(taskScheduler, taskSet, 1) { + extends TaskSetManager(clusterScheduler, taskSet, 0) { parent = null weight = 1 @@ -130,8 +130,8 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging } test("FIFO Scheduler Test") { - sc = new SparkContext("local", "TaskSchedulerSuite") - val taskScheduler = new ClusterScheduler(sc) + sc = new SparkContext("local", "ClusterSchedulerSuite") + val clusterScheduler = new ClusterScheduler(sc) var tasks = ArrayBuffer[Task[_]]() val task = new FakeTask(0) tasks += task @@ -141,9 +141,9 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) schedulableBuilder.buildPools() - val taskSetManager0 = createDummyTaskSetManager(0, 0, 2, taskScheduler, taskSet) - val taskSetManager1 = createDummyTaskSetManager(0, 1, 2, taskScheduler, taskSet) - val taskSetManager2 = createDummyTaskSetManager(0, 2, 2, taskScheduler, taskSet) + val taskSetManager0 = createDummyTaskSetManager(0, 0, 2, clusterScheduler, taskSet) + val taskSetManager1 = createDummyTaskSetManager(0, 1, 2, clusterScheduler, taskSet) + val taskSetManager2 = createDummyTaskSetManager(0, 2, 2, clusterScheduler, taskSet) schedulableBuilder.addTaskSetManager(taskSetManager0, null) schedulableBuilder.addTaskSetManager(taskSetManager1, null) schedulableBuilder.addTaskSetManager(taskSetManager2, null) @@ -157,8 +157,8 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging } test("Fair Scheduler Test") { - sc = new SparkContext("local", "TaskSchedulerSuite") - val taskScheduler = new ClusterScheduler(sc) + sc = new SparkContext("local", "ClusterSchedulerSuite") + val clusterScheduler = new ClusterScheduler(sc) var tasks = ArrayBuffer[Task[_]]() val task = new FakeTask(0) tasks += task @@ -186,15 +186,15 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging val properties2 = new Properties() properties2.setProperty("spark.scheduler.pool","2") - val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, taskScheduler, taskSet) - val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, taskScheduler, taskSet) - val taskSetManager12 = createDummyTaskSetManager(1, 2, 2, taskScheduler, taskSet) + val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, clusterScheduler, taskSet) + val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, clusterScheduler, taskSet) + val taskSetManager12 = createDummyTaskSetManager(1, 2, 2, clusterScheduler, taskSet) schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) - val taskSetManager23 = createDummyTaskSetManager(2, 3, 2, taskScheduler, taskSet) - val taskSetManager24 = createDummyTaskSetManager(2, 4, 2, taskScheduler, taskSet) + val taskSetManager23 = createDummyTaskSetManager(2, 3, 2, clusterScheduler, taskSet) + val taskSetManager24 = createDummyTaskSetManager(2, 4, 2, clusterScheduler, taskSet) schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) @@ -214,8 +214,8 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging } test("Nested Pool Test") { - sc = new SparkContext("local", "TaskSchedulerSuite") - val taskScheduler = new ClusterScheduler(sc) + sc = new SparkContext("local", "ClusterSchedulerSuite") + val clusterScheduler = new ClusterScheduler(sc) var tasks = ArrayBuffer[Task[_]]() val task = new FakeTask(0) tasks += task @@ -237,23 +237,23 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging pool1.addSchedulable(pool10) pool1.addSchedulable(pool11) - val taskSetManager000 = createDummyTaskSetManager(0, 0, 5, taskScheduler, taskSet) - val taskSetManager001 = createDummyTaskSetManager(0, 1, 5, taskScheduler, taskSet) + val taskSetManager000 = createDummyTaskSetManager(0, 0, 5, clusterScheduler, taskSet) + val taskSetManager001 = createDummyTaskSetManager(0, 1, 5, clusterScheduler, taskSet) pool00.addSchedulable(taskSetManager000) pool00.addSchedulable(taskSetManager001) - val taskSetManager010 = createDummyTaskSetManager(1, 2, 5, taskScheduler, taskSet) - val taskSetManager011 = createDummyTaskSetManager(1, 3, 5, taskScheduler, taskSet) + val taskSetManager010 = createDummyTaskSetManager(1, 2, 5, clusterScheduler, taskSet) + val taskSetManager011 = createDummyTaskSetManager(1, 3, 5, clusterScheduler, taskSet) pool01.addSchedulable(taskSetManager010) pool01.addSchedulable(taskSetManager011) - val taskSetManager100 = createDummyTaskSetManager(2, 4, 5, taskScheduler, taskSet) - val taskSetManager101 = createDummyTaskSetManager(2, 5, 5, taskScheduler, taskSet) + val taskSetManager100 = createDummyTaskSetManager(2, 4, 5, clusterScheduler, taskSet) + val taskSetManager101 = createDummyTaskSetManager(2, 5, 5, clusterScheduler, taskSet) pool10.addSchedulable(taskSetManager100) pool10.addSchedulable(taskSetManager101) - val taskSetManager110 = createDummyTaskSetManager(3, 6, 5, taskScheduler, taskSet) - val taskSetManager111 = createDummyTaskSetManager(3, 7, 5, taskScheduler, taskSet) + val taskSetManager110 = createDummyTaskSetManager(3, 6, 5, clusterScheduler, taskSet) + val taskSetManager111 = createDummyTaskSetManager(3, 7, 5, clusterScheduler, taskSet) pool11.addSchedulable(taskSetManager110) pool11.addSchedulable(taskSetManager111) 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 24689a7093..00f2fdd657 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -33,25 +33,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} -/** - * TaskScheduler that records the task sets that the DAGScheduler requested executed. - */ -class TaskSetRecordingTaskScheduler(sc: SparkContext, - mapOutputTrackerMaster: MapOutputTrackerMaster) extends ClusterScheduler(sc) { - /** Set of TaskSets the DAGScheduler has requested executed. */ - val taskSets = scala.collection.mutable.Buffer[TaskSet]() - override def start() = {} - override def stop() = {} - override def submitTasks(taskSet: TaskSet) = { - // normally done by TaskSetManager - taskSet.tasks.foreach(_.epoch = mapOutputTrackerMaster.getEpoch) - taskSets += taskSet - } - override def cancelTasks(stageId: Int) {} - override def setDAGScheduler(dagScheduler: DAGScheduler) = {} - override def defaultParallelism() = 2 -} - /** * 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 @@ -65,7 +46,24 @@ class TaskSetRecordingTaskScheduler(sc: SparkContext, * and capturing the resulting TaskSets from the mock TaskScheduler. */ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { - var taskScheduler: TaskSetRecordingTaskScheduler = null + + /** 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) = { + // normally done by TaskSetManager + taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) + taskSets += taskSet + } + override def cancelTasks(stageId: Int) {} + override def setDAGScheduler(dagScheduler: DAGScheduler) = {} + override def defaultParallelism() = 2 + } + var mapOutputTracker: MapOutputTrackerMaster = null var scheduler: DAGScheduler = null @@ -98,11 +96,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont before { sc = new SparkContext("local", "DAGSchedulerSuite") - mapOutputTracker = new MapOutputTrackerMaster() - taskScheduler = new TaskSetRecordingTaskScheduler(sc, mapOutputTracker) - taskScheduler.taskSets.clear() + taskSets.clear() cacheLocations.clear() results.clear() + mapOutputTracker = new MapOutputTrackerMaster() scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) { override def runLocally(job: ActiveJob) { // don't bother with the thread while unit testing @@ -207,7 +204,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont test("run trivial job") { val rdd = makeRdd(1, Nil) submit(rdd, Array(0)) - complete(taskScheduler.taskSets(0), List((Success, 42))) + complete(taskSets(0), List((Success, 42))) assert(results === Map(0 -> 42)) } @@ -228,7 +225,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val baseRdd = makeRdd(1, Nil) val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) submit(finalRdd, Array(0)) - complete(taskScheduler.taskSets(0), Seq((Success, 42))) + complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) } @@ -238,7 +235,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont cacheLocations(baseRdd.id -> 0) = Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")) submit(finalRdd, Array(0)) - val taskSet = taskScheduler.taskSets(0) + val taskSet = taskSets(0) assertLocations(taskSet, Seq(Seq("hostA", "hostB"))) complete(taskSet, Seq((Success, 42))) assert(results === Map(0 -> 42)) @@ -246,7 +243,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont test("trivial job failure") { submit(makeRdd(1, Nil), Array(0)) - failed(taskScheduler.taskSets(0), "some failure") + failed(taskSets(0), "some failure") assert(failure.getMessage === "Job aborted: some failure") } @@ -256,12 +253,12 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(1, List(shuffleDep)) submit(reduceRdd, Array(0)) - complete(taskScheduler.taskSets(0), Seq( + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) - complete(taskScheduler.taskSets(1), Seq((Success, 42))) + complete(taskSets(1), Seq((Success, 42))) assert(results === Map(0 -> 42)) } @@ -271,11 +268,11 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val shuffleId = shuffleDep.shuffleId val reduceRdd = makeRdd(2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) - complete(taskScheduler.taskSets(0), Seq( + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) // the 2nd ResultTask failed - complete(taskScheduler.taskSets(1), Seq( + complete(taskSets(1), Seq( (Success, 42), (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), null))) // this will get called @@ -283,10 +280,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont // ask the scheduler to try it again scheduler.resubmitFailedStages() // have the 2nd attempt pass - complete(taskScheduler.taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) + complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) // we can see both result blocks now assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === Array("hostA", "hostB")) - complete(taskScheduler.taskSets(3), Seq((Success, 43))) + complete(taskSets(3), Seq((Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) } @@ -302,7 +299,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val newEpoch = mapOutputTracker.getEpoch assert(newEpoch > oldEpoch) val noAccum = Map[Long, Any]() - val taskSet = taskScheduler.taskSets(0) + val taskSet = taskSets(0) // 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 non-failed host @@ -314,7 +311,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum, null, null)) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) - complete(taskScheduler.taskSets(1), Seq((Success, 42), (Success, 43))) + complete(taskSets(1), Seq((Success, 42), (Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) } @@ -329,14 +326,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runEvent(ExecutorLost("exec-hostA")) // DAGScheduler will immediately resubmit the stage after it appears to have no pending tasks // rather than marking it is as failed and waiting. - complete(taskScheduler.taskSets(0), Seq( + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) // have hostC complete the resubmitted task - complete(taskScheduler.taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) + complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) - complete(taskScheduler.taskSets(2), Seq((Success, 42))) + complete(taskSets(2), Seq((Success, 42))) assert(results === Map(0 -> 42)) } @@ -348,23 +345,23 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val finalRdd = makeRdd(1, List(shuffleDepTwo)) submit(finalRdd, Array(0)) // have the first stage complete normally - complete(taskScheduler.taskSets(0), Seq( + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 2)), (Success, makeMapStatus("hostB", 2)))) // have the second stage complete normally - complete(taskScheduler.taskSets(1), Seq( + complete(taskSets(1), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostC", 1)))) // fail the third stage because hostA went down - complete(taskScheduler.taskSets(2), Seq( + complete(taskSets(2), Seq( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // have DAGScheduler try again scheduler.resubmitFailedStages() - complete(taskScheduler.taskSets(3), Seq((Success, makeMapStatus("hostA", 2)))) - complete(taskScheduler.taskSets(4), Seq((Success, makeMapStatus("hostA", 1)))) - complete(taskScheduler.taskSets(5), Seq((Success, 42))) + complete(taskSets(3), Seq((Success, makeMapStatus("hostA", 2)))) + complete(taskSets(4), Seq((Success, makeMapStatus("hostA", 1)))) + complete(taskSets(5), Seq((Success, 42))) assert(results === Map(0 -> 42)) } @@ -378,24 +375,24 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) // complete stage 2 - complete(taskScheduler.taskSets(0), Seq( + complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 2)), (Success, makeMapStatus("hostB", 2)))) // complete stage 1 - complete(taskScheduler.taskSets(1), Seq( + complete(taskSets(1), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) // pretend stage 0 failed because hostA went down - complete(taskScheduler.taskSets(2), Seq( + complete(taskSets(2), Seq( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun. scheduler.resubmitFailedStages() - assertLocations(taskScheduler.taskSets(3), Seq(Seq("hostD"))) + assertLocations(taskSets(3), Seq(Seq("hostD"))) // allow hostD to recover - complete(taskScheduler.taskSets(3), Seq((Success, makeMapStatus("hostD", 1)))) - complete(taskScheduler.taskSets(4), Seq((Success, 42))) + complete(taskSets(3), Seq((Success, makeMapStatus("hostD", 1)))) + complete(taskSets(4), Seq((Success, 42))) assert(results === Map(0 -> 42)) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 2ac2d7a36a..b0d1902c67 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -64,20 +64,18 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA System.setProperty("spark.akka.frameSize", "1") } - before { - sc = new SparkContext("local", "test") - } - override def afterAll { System.clearProperty("spark.akka.frameSize") } test("handling results smaller than Akka frame size") { + sc = new SparkContext("local", "test") val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x) assert(result === 2) } - test("handling results larger than Akka frame size") { + test("handling results larger than Akka frame size") { + sc = new SparkContext("local", "test") val akkaFrameSize = sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) @@ -89,13 +87,16 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA } test("task retried if result missing from block manager") { + // Set the maximum number of task failures to > 0, so that the task set isn't aborted + // after the result is missing. + sc = new SparkContext("local[1,1]", "test") // If this test hangs, it's probably because no resource offers were made after the task // failed. val scheduler: ClusterScheduler = sc.taskScheduler match { case clusterScheduler: ClusterScheduler => clusterScheduler case _ => - assert(false, "Expect local cluster to use TaskScheduler") + assert(false, "Expect local cluster to use ClusterScheduler") throw new ClassCastException } scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 592bb11364..4bbb51532d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.executor.TaskMetrics import java.nio.ByteBuffer import org.apache.spark.util.{Utils, FakeClock} -class FakeDAGScheduler(taskScheduler: FakeTaskScheduler) extends DAGScheduler(taskScheduler) { +class FakeDAGScheduler(taskScheduler: FakeClusterScheduler) extends DAGScheduler(taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo) { taskScheduler.startedTasks += taskInfo.index } @@ -52,12 +52,12 @@ class FakeDAGScheduler(taskScheduler: FakeTaskScheduler) extends DAGScheduler(ta } /** - * A mock TaskScheduler implementation that just remembers information about tasks started and + * 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 TaskSetManager. */ -class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* execId, host */) +class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* execId, host */) extends ClusterScheduler(sc) { val startedTasks = new ArrayBuffer[Long] @@ -86,7 +86,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("TaskSet with no preferences") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + val sched = new FakeClusterScheduler(sc, ("exec1", "host1")) val taskSet = createTaskSet(1) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) @@ -112,7 +112,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("multiple offers with no preferences") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + val sched = new FakeClusterScheduler(sc, ("exec1", "host1")) val taskSet = createTaskSet(3) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) @@ -143,7 +143,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("basic delay scheduling") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + val sched = new FakeClusterScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) val taskSet = createTaskSet(4, Seq(TaskLocation("host1", "exec1")), Seq(TaskLocation("host2", "exec2")), @@ -187,7 +187,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("delay scheduling with fallback") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, + val sched = new FakeClusterScheduler(sc, ("exec1", "host1"), ("exec2", "host2"), ("exec3", "host3")) val taskSet = createTaskSet(5, Seq(TaskLocation("host1")), @@ -227,7 +227,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("delay scheduling with failed hosts") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + val sched = new FakeClusterScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) val taskSet = createTaskSet(3, Seq(TaskLocation("host1")), Seq(TaskLocation("host2")), @@ -259,7 +259,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("task result lost") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + val sched = new FakeClusterScheduler(sc, ("exec1", "host1")) val taskSet = createTaskSet(1) val clock = new FakeClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) @@ -276,7 +276,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("repeated failures lead to task set abortion") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + val sched = new FakeClusterScheduler(sc, ("exec1", "host1")) val taskSet = createTaskSet(1) val clock = new FakeClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala index e873400680..4e988b8017 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -21,16 +21,16 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark._ import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler} -import org.apache.spark.scheduler.TaskScheduler +import org.apache.spark.scheduler.ClusterScheduler import org.apache.spark.util.Utils /** * - * This is a simple extension to TaskScheduler - to ensure that appropriate initialization of + * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of * ApplicationMaster, etc. is done */ private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) - extends TaskScheduler(sc) { + extends ClusterScheduler(sc) { logInfo("Created YarnClusterScheduler") -- cgit v1.2.3 From c64690d7252248df97bbe4b2bef8f540b977842d Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Thu, 14 Nov 2013 09:34:56 -0800 Subject: Changed local backend to use Akka actor --- .../spark/scheduler/local/LocalBackend.scala | 80 +++++++++++++++------- 1 file changed, 57 insertions(+), 23 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 3e9d31cd5e..d9b941d694 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -21,21 +21,26 @@ import java.nio.ByteBuffer import akka.actor.{Actor, ActorRef, Props} -import org.apache.spark.{SparkContext, SparkEnv, TaskState} +import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} import org.apache.spark.scheduler.{SchedulerBackend, ClusterScheduler, WorkerOffer} +private case class ReviveOffers() + +private case class StatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) + +private case class KillTask(taskId: Long) + /** - * LocalBackend is used when running a local version of Spark where the executor, backend, and - * master all run in the same JVM. It sits behind a ClusterScheduler and handles launching tasks - * on a single Executor (created by the LocalBackend) running locally. - * - * THREADING: Because methods can be called both from the Executor and the TaskScheduler, and - * because the Executor class is not thread safe, all methods are synchronized. + * Calls to LocalBackend are all serialized through LocalActor. Using an actor makes the calls on + * LocalBackend asynchronous, which is necessary to prevent deadlock between LocalBackend + * and the ClusterScheduler. */ -private[spark] class LocalBackend(scheduler: ClusterScheduler, private val totalCores: Int) - extends SchedulerBackend with ExecutorBackend { +private[spark] class LocalActor( + scheduler: ClusterScheduler, + executorBackend: LocalBackend, + private val totalCores: Int) extends Actor with Logging { private var freeCores = totalCores @@ -44,31 +49,60 @@ private[spark] class LocalBackend(scheduler: ClusterScheduler, private val total val executor = new Executor(localExecutorId, localExecutorHostname, Seq.empty, isLocal = true) - override def start() { - } + def receive = { + case ReviveOffers => + reviveOffers() - override def stop() { + case StatusUpdate(taskId, state, serializedData) => + scheduler.statusUpdate(taskId, state, serializedData) + if (TaskState.isFinished(state)) { + freeCores += 1 + reviveOffers() + } + + case KillTask(taskId) => + executor.killTask(taskId) } - override def reviveOffers() = synchronized { - val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) + def reviveOffers() { + val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) for (task <- scheduler.resourceOffers(offers).flatten) { freeCores -= 1 - executor.launchTask(this, task.taskId, task.serializedTask) + executor.launchTask(executorBackend, task.taskId, task.serializedTask) } } +} + +/** + * LocalBackend is used when running a local version of Spark where the executor, backend, and + * master all run in the same JVM. It sits behind a ClusterScheduler and handles launching tasks + * on a single Executor (created by the LocalBackend) running locally. + */ +private[spark] class LocalBackend(scheduler: ClusterScheduler, private val totalCores: Int) + extends SchedulerBackend with ExecutorBackend { + + var localActor: ActorRef = null + + override def start() { + localActor = SparkEnv.get.actorSystem.actorOf( + Props(new LocalActor(scheduler, this, totalCores)), + "LocalBackendActor") + } + + override def stop() { + } + + override def reviveOffers() { + localActor ! ReviveOffers + } override def defaultParallelism() = totalCores - override def killTask(taskId: Long, executorId: String) = synchronized { - executor.killTask(taskId) + override def killTask(taskId: Long, executorId: String) { + localActor ! KillTask(taskId) } - override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) = synchronized { - scheduler.statusUpdate(taskId, state, serializedData) - if (TaskState.isFinished(state)) { - freeCores += 1 - reviveOffers() - } + override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) { + localActor ! StatusUpdate(taskId, state, serializedData) } } -- cgit v1.2.3 From 2b807e4f2f853a9b1e8cba5147d182e7b05022bc Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Thu, 14 Nov 2013 13:33:11 -0800 Subject: Fix bug where scheduler could hang after task failure. When a task fails, we need to call reviveOffers() so that the task can be rescheduled on a different machine. In the current code, the state in ClusterTaskSetManager indicating which tasks are pending may be updated after revive offers is called (there's a race condition here), so when revive offers is called, the task set manager does not yet realize that there are failed tasks that need to be relaunched. --- .../scala/org/apache/spark/scheduler/ClusterScheduler.scala | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala index 37d554715d..2e4ba53d9b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala @@ -250,7 +250,6 @@ private[spark] class ClusterScheduler( def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { var failedExecutor: Option[String] = None - var taskFailed = false synchronized { try { if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) { @@ -270,9 +269,6 @@ private[spark] class ClusterScheduler( } taskIdToExecutorId.remove(tid) } - if (state == TaskState.FAILED) { - taskFailed = true - } activeTaskSets.get(taskSetId).foreach { taskSet => if (state == TaskState.FINISHED) { taskSet.removeRunningTask(tid) @@ -294,10 +290,6 @@ private[spark] class ClusterScheduler( dagScheduler.executorLost(failedExecutor.get) backend.reviveOffers() } - if (taskFailed) { - // Also revive offers if a task had failed for some reason other than host lost - backend.reviveOffers() - } } def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long) { @@ -317,8 +309,9 @@ private[spark] class ClusterScheduler( taskState: TaskState, reason: Option[TaskEndReason]) = synchronized { taskSetManager.handleFailedTask(tid, taskState, reason) - if (taskState == TaskState.FINISHED) { - // The task finished successfully but the result was lost, so we should revive offers. + if (taskState != TaskState.KILLED) { + // Need to revive offers again now that the task set manager state has been updated to + // reflect failed tasks that need to be re-run. backend.reviveOffers() } } -- cgit v1.2.3 From 52144caaa70363ffcc63e1f52db32eb1654c1213 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Thu, 14 Nov 2013 14:56:53 -0800 Subject: Don't retry tasks if result wasn't serializable --- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index bc35e53220..e3929e61ac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -29,6 +29,7 @@ import org.apache.spark.{ExceptionFailure, FetchFailed, Logging, Resubmitted, Sp Success, TaskEndReason, TaskKilled, TaskResultLost, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.util.{SystemClock, Clock} +import java.io.NotSerializableException /** @@ -488,7 +489,16 @@ private[spark] class TaskSetManager( return case ef: ExceptionFailure => - sched.dagScheduler.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null)) + sched.dagScheduler.taskEnded( + tasks(index), ef, null, null, info, ef.metrics.getOrElse(null)) + if (ef.className == classOf[NotSerializableException].getName()) { + // If the task result wasn't rerializable, there's no point in trying to re-execute it. + logError("Task %s:%s had a not serializable result: %s; not retrying".format( + taskSet.id, index, ef.description)) + abort("Task %s:%s had a not serializable result: %s".format( + taskSet.id, index, ef.description)) + return + } val key = ef.description val now = clock.getTime() val (printFull, dupCount) = { -- cgit v1.2.3 From d4cd32330e1e4ac83b38bc922a9d3fd85f85f606 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Thu, 14 Nov 2013 13:57:02 +0800 Subject: Some fixes for previous master merge commits --- core/src/main/scala/org/apache/spark/deploy/client/Client.scala | 2 +- core/src/main/scala/org/apache/spark/deploy/master/Master.scala | 3 ++- core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 4 +--- project/SparkBuild.scala | 1 + 4 files changed, 5 insertions(+), 5 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index be8693ec54..3953a3e178 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -21,7 +21,6 @@ import java.util.concurrent.TimeoutException import scala.concurrent.duration._ import scala.concurrent.Await -import scala.concurrent.ExecutionContext.Implicits.global import akka.actor._ import akka.actor.Terminated @@ -84,6 +83,7 @@ private[spark] class Client( def registerWithMaster() { tryRegisterAllMasters() + import context.dispatcher var retries = 0 lazy val retryTimer: Cancellable = context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { 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 26f980760d..0545ad185f 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 @@ -24,7 +24,6 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ import scala.concurrent.duration.{ Duration, FiniteDuration } -import scala.concurrent.ExecutionContext.Implicits.global import akka.actor._ import akka.pattern.ask @@ -58,6 +57,8 @@ import java.util.concurrent.TimeUnit private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { + import context.dispatcher + val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retainedApplications", "200").toInt 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 991b22d9f8..98c57ca0b0 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 @@ -23,7 +23,6 @@ import java.io.File import scala.collection.mutable.HashMap import scala.concurrent.duration._ -import scala.concurrent.ExecutionContext.Implicits.global import akka.actor._ import akka.remote.{RemotingLifecycleEvent, AssociationErrorEvent, DisassociatedEvent} @@ -61,6 +60,7 @@ private[spark] class Worker( masterUrls: Array[String], workDirPath: String = null) extends Actor with Logging { + import context.dispatcher Utils.checkHost(host, "Expected hostname") assert (port > 0) @@ -175,8 +175,6 @@ private[spark] class Worker( retryTimer // start timer } - import context.dispatcher - override def receive = { case RegisteredWorker(masterUrl, masterWebUiUrl) => logInfo("Successfully registered with master " + masterUrl) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9a3cbbe7d2..26e6a8326c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -227,6 +227,7 @@ object SparkBuild extends Build { "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib), "org.apache.avro" % "avro" % "1.7.4", "org.apache.avro" % "avro-ipc" % "1.7.4" excludeAll(excludeNetty), + "org.apache.zookeeper" % "zookeeper" % "3.4.5" 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", -- cgit v1.2.3 From f629ba95b6a1a3508463bfdcb03efcfaa3327cb5 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 14 Nov 2013 22:13:09 -0800 Subject: Various merge corrections I've diff'd this patch against my own -- since they were both created independently, this means that two sets of eyes have gone over all the merge conflicts that were created, so I'm feeling significantly more confident in the resulting PR. @rxin has looked at the changes to the repl and is resoundingly confident that they are correct. --- .../apache/spark/api/java/function/Function.java | 2 - .../apache/spark/api/java/function/Function2.java | 2 - .../org/apache/spark/deploy/client/Client.scala | 4 +- .../spark/deploy/master/ApplicationState.scala | 3 +- .../org/apache/spark/deploy/master/Master.scala | 14 +---- .../org/apache/spark/deploy/worker/Worker.scala | 17 +------ .../executor/CoarseGrainedExecutorBackend.scala | 1 - .../scala/org/apache/spark/executor/Executor.scala | 6 ++- .../org/apache/spark/rdd/AsyncRDDActions.scala | 2 +- .../main/scala/org/apache/spark/rdd/BlockRDD.scala | 3 +- .../scala/org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../org/apache/spark/scheduler/DAGScheduler.scala | 7 +-- .../spark/scheduler/cluster/ClusterScheduler.scala | 2 +- .../mesos/CoarseMesosSchedulerBackend.scala | 1 + .../main/scala/org/apache/spark/util/Utils.scala | 5 +- .../apache/spark/util/collection/OpenHashMap.scala | 1 - docs/hadoop-third-party-distributions.md | 4 +- project/SparkBuild.scala | 11 +--- .../scala/org/apache/spark/repl/SparkILoop.scala | 14 ++--- .../scala/org/apache/spark/repl/ReplSuite.scala | 2 +- .../spark/streaming/NetworkInputTracker.scala | 2 +- .../spark/streaming/PairDStreamFunctions.scala | 1 - .../spark/streaming/api/java/JavaPairDStream.scala | 2 +- .../spark/streaming/dstream/CoGroupedDStream.scala | 59 ---------------------- .../streaming/dstream/KafkaInputDStream.scala | 1 - 25 files changed, 33 insertions(+), 135 deletions(-) delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function.java b/core/src/main/scala/org/apache/spark/api/java/function/Function.java index 49e661a376..537439ef53 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function.java @@ -29,8 +29,6 @@ import java.io.Serializable; * when mapping RDDs of other types. */ public abstract class Function extends WrappedFunction1 implements Serializable { - public abstract R call(T t) throws Exception; - public ClassTag returnType() { return ClassTag$.MODULE$.apply(Object.class); } diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java index cf77bb6b73..a2d1214fb4 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java @@ -28,8 +28,6 @@ import java.io.Serializable; public abstract class Function2 extends WrappedFunction2 implements Serializable { - public abstract R call(T1 t1, T2 t2) throws Exception; - public ClassTag returnType() { return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 3953a3e178..572fc347df 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -145,11 +145,11 @@ private[spark] class Client( markDisconnected() case DisassociatedEvent(_, address, _) if address == masterAddress => - logError("Connection to master failed; stopping client") + logWarning("Connection to master failed; waiting for master to reconnect...") markDisconnected() case AssociationErrorEvent(_, _, address, _) if address == masterAddress => - logError("Connection to master failed; stopping client") + logWarning("Connection to master failed; waiting for master to reconnect...") markDisconnected() case StopClient => diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala index a74d7be4c9..67e6c5d66a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala @@ -17,8 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object ApplicationState - extends Enumeration { +private[spark] object ApplicationState extends Enumeration { type ApplicationState = Value 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 0545ad185f..7db5097c2d 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 @@ -23,7 +23,7 @@ import java.text.SimpleDateFormat import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ -import scala.concurrent.duration.{ Duration, FiniteDuration } +import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor._ import akka.pattern.ask @@ -41,16 +41,6 @@ import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed import org.apache.spark.deploy.DeployMessages.KillExecutor import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged import scala.Some -import org.apache.spark.deploy.DeployMessages.LaunchExecutor -import org.apache.spark.deploy.DeployMessages.RegisteredApplication -import org.apache.spark.deploy.DeployMessages.RegisterWorker -import org.apache.spark.deploy.DeployMessages.ExecutorUpdated -import org.apache.spark.deploy.DeployMessages.MasterStateResponse -import org.apache.spark.deploy.DeployMessages.ExecutorAdded -import org.apache.spark.deploy.DeployMessages.RegisterApplication -import org.apache.spark.deploy.DeployMessages.ApplicationRemoved -import org.apache.spark.deploy.DeployMessages.Heartbeat -import org.apache.spark.deploy.DeployMessages.RegisteredWorker import akka.actor.Terminated import akka.serialization.SerializationExtension import java.util.concurrent.TimeUnit @@ -571,7 +561,7 @@ 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(classOf[Master], host, boundPort, webUiPort), name = actorName) - val timeoutDuration : FiniteDuration = Duration.create( + val timeoutDuration: FiniteDuration = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, TimeUnit.SECONDS) implicit val timeout = Timeout(timeoutDuration) val respFuture = actor ? RequestWebUIPort // ask pattern 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 98c57ca0b0..07189ac850 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 @@ -25,7 +25,7 @@ import scala.collection.mutable.HashMap import scala.concurrent.duration._ import akka.actor._ -import akka.remote.{RemotingLifecycleEvent, AssociationErrorEvent, DisassociatedEvent} +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import org.apache.spark.Logging import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} @@ -34,19 +34,6 @@ 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.{Utils, AkkaUtils} -import org.apache.spark.deploy.DeployMessages.WorkerStateResponse -import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed -import org.apache.spark.deploy.DeployMessages.KillExecutor -import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged -import scala.Some -import akka.remote.DisassociatedEvent -import org.apache.spark.deploy.DeployMessages.LaunchExecutor -import org.apache.spark.deploy.DeployMessages.RegisterWorker -import org.apache.spark.deploy.DeployMessages.WorkerSchedulerStateResponse -import org.apache.spark.deploy.DeployMessages.MasterChanged -import org.apache.spark.deploy.DeployMessages.Heartbeat -import org.apache.spark.deploy.DeployMessages.RegisteredWorker -import akka.actor.Terminated /** * @param masterUrls Each url should look like spark://host:port. @@ -248,7 +235,7 @@ private[spark] class Worker( } } - case DisassociatedEvent(_, _, _) => + case DisassociatedEvent(_, address, _) if address == master.path.address => masterDisconnected() case RequestWorkerState => { diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 73fa7d6b6a..50302fcca4 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -107,7 +107,6 @@ private[spark] object CoarseGrainedExecutorBackend { // set it val sparkHostPort = hostname + ":" + boundPort System.setProperty("spark.hostPort", sparkHostPort) - actorSystem.actorOf( Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores), name = "Executor") 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 de4540493a..0b0a60ee60 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -118,7 +118,11 @@ private[spark] class Executor( } } - private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size") + // Akka's message frame size. If task result is bigger than this, we use the block manager + // to send the result back. + private val akkaFrameSize = { + env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size") + } // Start worker thread pool val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker") diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index 44c5078621..d1c74a5063 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -21,9 +21,9 @@ import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable.ArrayBuffer import scala.concurrent.ExecutionContext.Implicits.global +import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} -import scala.reflect.ClassTag /** * A set of asynchronous RDD actions available through an implicit conversion. 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 63b9fe1478..424354ae16 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -17,9 +17,10 @@ package org.apache.spark.rdd +import scala.reflect.ClassTag + import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} import org.apache.spark.storage.{BlockId, BlockManager} -import scala.reflect.ClassTag private[spark] class BlockRDDPartition(val blockId: BlockId, idx: Int) extends Partition { val index = idx 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 47e958b5e6..53f77a38f5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -52,7 +52,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp * sources in HBase, or S3). * * @param sc The SparkContext to associate the RDD with. - * @param broadCastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed + * @param broadcastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed * variabe references an instance of JobConf, then that JobConf will be used for the Hadoop job. * Otherwise, a new JobConf will be created on each slave using the enclosed Configuration. * @param initLocalJobConfFuncOpt Optional closure used to initialize any JobConf that HadoopRDD 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 7b4fc6b9be..fdea3f6f88 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -20,13 +20,14 @@ package org.apache.spark.scheduler import java.io.NotSerializableException import java.util.Properties import java.util.concurrent.atomic.AtomicInteger -import scala.concurrent.duration._ -import scala.concurrent.ExecutionContext.Implicits.global -import akka.actor._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration._ import scala.reflect.ClassTag +import akka.actor._ + 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/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala index 2d8a0a62c9..9975ec1ab6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala @@ -25,8 +25,8 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet -import scala.concurrent.duration._ import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration._ import org.apache.spark._ import org.apache.spark.TaskState.TaskState diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 8de9b72b2f..84fe3094cc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -181,6 +181,7 @@ private[spark] class CoarseMesosSchedulerBackend( !slaveIdsWithExecutors.contains(slaveId)) { // Launch an executor on the slave val cpusToUse = math.min(cpus, maxCores - totalCoresAcquired) + totalCoresAcquired += cpusToUse val taskId = newMesosTaskId() taskIdToSlaveId(taskId) = slaveId slaveIdsWithExecutors += slaveId diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 7557ddab19..02adcb41c6 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -22,14 +22,11 @@ import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address} import java.util.{Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} - -import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.collection.Map +import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag -import scala.Some - import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index 45849b3380..c26f23d500 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -19,7 +19,6 @@ package org.apache.spark.util.collection import scala.reflect.ClassTag - /** * A fast hash map implementation for nullable keys. This hash map supports insertions and updates, * but not deletions. This map is about 5X faster than java.util.HashMap, while using much less diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index f706625fe9..b33af2cf24 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -25,8 +25,8 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors.

CDH Releases

- - + + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 26e6a8326c..476e7c5800 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -105,12 +105,6 @@ object SparkBuild extends Build { // also check the local Maven repository ~/.m2 resolvers ++= Seq(Resolver.file("Local Maven Repo", file(Path.userHome + "/.m2/repository"))), - // Shared between both core and streaming. - resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"), - - // Shared between both examples and streaming. - resolvers ++= Seq("Mqtt Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/"), - // For Sonatype publishing resolvers ++= Seq("sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/"), @@ -292,11 +286,10 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy), - "com.sksamuel.kafka" %% "kafka" % "0.8.0-beta1" + "org.apache.kafka" % "kafka_2.9.2" % "0.8.0-beta1" exclude("com.sun.jdmk", "jmxtools") exclude("com.sun.jmx", "jmxri") - exclude("net.sf.jopt-simple", "jopt-simple") - excludeAll(excludeNetty), + exclude("net.sf.jopt-simple", "jopt-simple"), "org.eclipse.paho" % "mqtt-client" % "0.4.0", "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 43e504c290..523fd1222d 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -940,17 +940,9 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, if (prop != null) prop else "local" } } - val jars = Option(System.getenv("ADD_JARS")).map(_.split(',')) - .getOrElse(new Array[String](0)) - .map(new java.io.File(_).getAbsolutePath) - try { - sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars) - } catch { - case e: Exception => - e.printStackTrace() - echo("Failed to create SparkContext, exiting...") - sys.exit(1) - } + 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 } diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 418c31e24b..c230a03298 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -78,7 +78,7 @@ class ReplSuite extends FunSuite { System.clearProperty("spark.hostPort") } - test ("simple foreach with accumulator") { + test("simple foreach with accumulator") { val output = runInterpreter("local", """ |val accum = sc.accumulator(0) |sc.parallelize(1 to 10).foreach(x => accum += x) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala index 66fe6e7870..6e9a781978 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala @@ -25,10 +25,10 @@ import org.apache.spark.SparkContext._ import scala.collection.mutable.HashMap import scala.collection.mutable.Queue +import scala.concurrent.duration._ import akka.actor._ import akka.pattern.ask -import scala.concurrent.duration._ import akka.dispatch._ import org.apache.spark.storage.BlockId 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 ea5c165691..80af96c060 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala @@ -33,7 +33,6 @@ import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.hadoop.mapred.OutputFormat import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.conf.Configuration -import scala.Some class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) extends Serializable { 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 3ba37bed4d..dfd6e27c3e 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 @@ -728,7 +728,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } object JavaPairDStream { - implicit def fromPairDStream[K: ClassTag, V: ClassTag](dstream: DStream[(K, V)]) : JavaPairDStream[K, V] = { + implicit def fromPairDStream[K: ClassTag, V: ClassTag](dstream: DStream[(K, V)]) = { new JavaPairDStream[K, V](dstream) } 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 deleted file mode 100644 index 16c1567355..0000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala +++ /dev/null @@ -1,59 +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.spark.streaming.dstream - -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} -import scala.reflect.ClassTag - -private[streaming] -class CoGroupedDStream[K : ClassTag]( - parents: Seq[DStream[(K, _)]], - partitioner: Partitioner - ) extends DStream[(K, Seq[Seq[_]])](parents.head.ssc) { - - if (parents.length == 0) { - throw new IllegalArgumentException("Empty array of parents") - } - - if (parents.map(_.ssc).distinct.size > 1) { - throw new IllegalArgumentException("Array of parents have different StreamingContexts") - } - - if (parents.map(_.slideDuration).distinct.size > 1) { - throw new IllegalArgumentException("Array of parents have different slide times") - } - - override def dependencies = parents.toList - - override def slideDuration: Duration = parents.head.slideDuration - - override def compute(validTime: Time): Option[RDD[(K, Seq[Seq[_]])]] = { - val part = partitioner - val rdds = parents.flatMap(_.getOrCompute(validTime)) - if (rdds.size > 0) { - val q = new CoGroupedRDD[K](rdds, part) - Some(q) - } else { - None - } - } - -} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala index ec0096c85f..526f5564c7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala @@ -33,7 +33,6 @@ import org.I0Itec.zkclient._ import scala.collection.Map import scala.reflect.ClassTag - /** * Input stream that pulls messages from a Kafka Broker. * -- cgit v1.2.3 From 2b0a6e7d9210ed828395243027c7001f7dae77a4 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Fri, 15 Nov 2013 18:34:28 -0800 Subject: Fixed error message in ClusterScheduler to be consistent with the old LocalScheduler --- .../main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index e3929e61ac..7989e6ab32 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -468,6 +468,7 @@ private[spark] class TaskSetManager( removeRunningTask(tid) val index = info.index info.markFailed() + var failureReason = "unknown" if (!successful(index)) { logWarning("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) copiesRunning(index) -= 1 @@ -500,6 +501,7 @@ private[spark] class TaskSetManager( return } val key = ef.description + failureReason = "Exception failure: %s".format(ef.description) val now = clock.getTime() val (printFull, dupCount) = { if (recentExceptions.contains(key)) { @@ -525,7 +527,8 @@ private[spark] class TaskSetManager( } case TaskResultLost => - logWarning("Lost result for TID %s on host %s".format(tid, info.host)) + failureReason = "Lost result for TID %s on host %s".format(tid, info.host) + logWarning(failureReason) sched.dagScheduler.taskEnded(tasks(index), TaskResultLost, null, null, info, null) case _ => {} @@ -537,7 +540,8 @@ private[spark] class TaskSetManager( if (numFailures(index) > maxTaskFailures) { logError("Task %s:%d failed more than %d times; aborting job".format( taskSet.id, index, maxTaskFailures)) - abort("Task %s:%d failed more than %d times".format(taskSet.id, index, maxTaskFailures)) + abort("Task %s:%d failed more than %d times (most recent failure: %s)".format( + taskSet.id, index, maxTaskFailures, failureReason)) } } } else { -- cgit v1.2.3 From 7de180fd13fda2e5d4486dfca9e2a9997ec7f4d0 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 18 Nov 2013 20:05:05 -0800 Subject: Remove explicit boxing --- core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') 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 b002468442..70f7f01d2b 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 @@ -169,9 +169,9 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav * If the RDD contains infinity, NaN throws an exception * If the elements in RDD do not vary (max == min) always returns a single bucket. */ - def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = { + def histogram(bucketCount: Int): Pair[Array[scala.Double], Array[Long]] = { val result = srdd.histogram(bucketCount) - (result._1.map(scala.Double.box(_)), result._2) + (result._1, result._2) } /** -- cgit v1.2.3 From e163e31c2003558d304ba5ac7b67361956037041 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 18 Nov 2013 20:13:25 -0800 Subject: Add spaces --- .../test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala | 14 ++++++++++++++ 1 file changed, 14 insertions(+) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index 0d8ac19024..7f50a5a47c 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -39,6 +39,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) } + test("WorksWithOutOfRangeWithOneBucket") { // Verify that if all of the elements are out of range the counts are zero val rdd = sc.parallelize(Seq(10.01, -0.01)) @@ -49,6 +50,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) } + test("WorksInRangeWithOneBucket") { // Verify the basic case of one bucket and all elements in that bucket works val rdd = sc.parallelize(Seq(1, 2, 3, 4)) @@ -59,6 +61,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) } + test("WorksInRangeWithOneBucketExactMatch") { // Verify the basic case of one bucket and all elements in that bucket works val rdd = sc.parallelize(Seq(1, 2, 3, 4)) @@ -69,6 +72,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) } + test("WorksWithOutOfRangeWithTwoBuckets") { // Verify that out of range works with two buckets val rdd = sc.parallelize(Seq(10.01, -0.01)) @@ -79,6 +83,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) } + test("WorksWithOutOfRangeWithTwoUnEvenBuckets") { // Verify that out of range works with two un even buckets val rdd = sc.parallelize(Seq(10.01, -0.01)) @@ -87,6 +92,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { val expectedHistogramResults = Array(0, 0) assert(histogramResults === expectedHistogramResults) } + test("WorksInRangeWithTwoBuckets") { // Make sure that it works with two equally spaced buckets and elements in each val rdd = sc.parallelize(Seq(1, 2, 3, 5, 6)) @@ -97,6 +103,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) } + test("WorksInRangeWithTwoBucketsAndNaN") { // Make sure that it works with two equally spaced buckets and elements in each val rdd = sc.parallelize(Seq(1, 2, 3, 5, 6, Double.NaN)) @@ -107,6 +114,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) } + test("WorksInRangeWithTwoUnevenBuckets") { // Make sure that it works with two unequally spaced buckets and elements in each val rdd = sc.parallelize(Seq(1, 2, 3, 5, 6)) @@ -115,6 +123,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { val expectedHistogramResults = Array(3, 2) assert(histogramResults === expectedHistogramResults) } + test("WorksMixedRangeWithTwoUnevenBuckets") { // Make sure that it works with two unequally spaced buckets and elements in each val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01)) @@ -123,6 +132,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { val expectedHistogramResults = Array(4, 3) assert(histogramResults === expectedHistogramResults) } + test("WorksMixedRangeWithFourUnevenBuckets") { // Make sure that it works with two unequally spaced buckets and elements in each val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, @@ -132,6 +142,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { val expectedHistogramResults = Array(4, 2, 1, 3) assert(histogramResults === expectedHistogramResults) } + test("WorksMixedRangeWithUnevenBucketsAndNaN") { // Make sure that it works with two unequally spaced buckets and elements in each val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, @@ -161,6 +172,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { val expectedHistogramResults = Array(4, 2, 1, 2, 4) assert(histogramResults === expectedHistogramResults) } + test("WorksWithOutOfRangeWithInfiniteBuckets") { // Verify that out of range works with two buckets val rdd = sc.parallelize(Seq(10.01, -0.01, Double.NaN)) @@ -214,6 +226,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramResults === expectedHistogramResults) assert(histogramBuckets === expectedHistogramBuckets) } + test("WorksWithoutBucketsBasicTwo") { // Verify the basic case of one bucket and all elements in that bucket works val rdd = sc.parallelize(Seq(1, 2, 3, 4)) @@ -223,6 +236,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramResults === expectedHistogramResults) assert(histogramBuckets === expectedHistogramBuckets) } + test("WorksWithoutBucketsWithMoreRequestedThanElements") { // Verify the basic case of one bucket and all elements in that bucket works val rdd = sc.parallelize(Seq(1, 2)) -- cgit v1.2.3 From fd031679df59b83ae0a735ea77c49623f6e257c4 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 21 Nov 2013 11:28:37 -0800 Subject: Added partitioner aware union, modified DStream.window. --- .../spark/rdd/PartitionerAwareUnionRDD.scala | 65 ++++++++++++++++++++++ .../test/scala/org/apache/spark/rdd/RDDSuite.scala | 27 +++++++++ .../spark/streaming/dstream/WindowedDStream.scala | 41 +------------- 3 files changed, 94 insertions(+), 39 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala new file mode 100644 index 0000000000..96cf93f99e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -0,0 +1,65 @@ +package org.apache.spark.rdd + +import org.apache.spark.{TaskContext, OneToOneDependency, SparkContext, Partition} + +private[spark] +class PartitionerAwareUnionRDDPartition(val idx: Int, val partitions: Array[Partition]) + extends Partition { + override val index = idx + override def hashCode(): Int = idx +} + +private[spark] +class PartitionerAwareUnionRDD[T: ClassManifest]( + sc: SparkContext, + var rdds: Seq[RDD[T]] + ) extends RDD[T](sc, rdds.map(x => new OneToOneDependency(x))) { + require(rdds.length > 0) + require(rdds.flatMap(_.partitioner).toSet.size == 1, + "Parent RDDs have different partitioners: " + rdds.flatMap(_.partitioner)) + + override val partitioner = rdds.head.partitioner + + override def getPartitions: Array[Partition] = { + val numPartitions = rdds.head.partitions.length + (0 until numPartitions).map(index => { + val parentPartitions = rdds.map(_.partitions(index)).toArray + new PartitionerAwareUnionRDDPartition(index, parentPartitions) + }).toArray + } + + // Get the location where most of the partitions of parent RDDs are located + override def getPreferredLocations(s: Partition): Seq[String] = { + logDebug("Getting preferred locations for " + this) + val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].partitions + val locations = rdds.zip(parentPartitions).flatMap { + case (rdd, part) => { + val parentLocations = currPrefLocs(rdd, part) + logDebug("Location of " + rdd + " partition " + part.index + " = " + parentLocations) + parentLocations + } + } + + if (locations.isEmpty) { + Seq.empty + } else { + Seq(locations.groupBy(x => x).map(x => (x._1, x._2.length)).maxBy(_._2)._1) + } + } + + override def compute(s: Partition, context: TaskContext): Iterator[T] = { + val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].partitions + rdds.zip(parentPartitions).iterator.flatMap { + case (rdd, p) => rdd.iterator(p, context) + } + } + + // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) + private def currPrefLocs(rdd: RDD[_], part: Partition): Seq[String] = { + rdd.context.getPreferredLocs(rdd, part.index).map(tl => tl.host) + } +} + + + + diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 354ab8ae5d..88b36a6855 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -71,6 +71,33 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(sc.union(Seq(nums, nums)).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) } + test("partitioner aware union") { + import SparkContext._ + def makeRDDWithPartitioner(seq: Seq[Int]) = { + sc.makeRDD(seq, 1) + .map(x => (x, null)) + .partitionBy(new HashPartitioner(2)) + .mapPartitions(_.map(_._1), true) + } + + val nums1 = makeRDDWithPartitioner(1 to 4) + val nums2 = makeRDDWithPartitioner(5 to 8) + assert(nums1.partitioner == nums2.partitioner) + assert(new PartitionerAwareUnionRDD(sc, Seq(nums1)).collect().toSet === Set(1, 2, 3, 4)) + + val union = new PartitionerAwareUnionRDD(sc, Seq(nums1, nums2)) + assert(union.collect().toSet === Set(1, 2, 3, 4, 5, 6, 7, 8)) + val nums1Parts = nums1.collectPartitions() + val nums2Parts = nums2.collectPartitions() + val unionParts = union.collectPartitions() + assert(nums1Parts.length === 2) + assert(nums2Parts.length === 2) + assert(unionParts.length === 2) + assert((nums1Parts(0) ++ nums2Parts(0)).toList === unionParts(0).toList) + assert((nums1Parts(1) ++ nums2Parts(1)).toList === unionParts(1).toList) + assert(union.partitioner === nums1.partitioner) + } + test("aggregate") { val pairs = sc.makeRDD(Array(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3))) type StringMap = HashMap[String, Int] 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 03f522e581..49f84310bc 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,8 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.UnionRDD +import org.apache.spark.rdd.{PartitionerAwareUnionRDD, RDD, UnionRDD} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark._ @@ -57,7 +56,7 @@ class WindowedDStream[T: ClassManifest]( val rddsInWindow = parent.slice(currentWindow) val windowRDD = if (rddsInWindow.flatMap(_.partitioner).distinct.length == 1) { logInfo("Using partition aware union") - new PartitionAwareUnionRDD(ssc.sc, rddsInWindow) + new PartitionerAwareUnionRDD(ssc.sc, rddsInWindow) } else { logInfo("Using normal union") new UnionRDD(ssc.sc,rddsInWindow) @@ -66,39 +65,3 @@ class WindowedDStream[T: ClassManifest]( } } -private[streaming] -class PartitionAwareUnionRDDPartition(val idx: Int, val partitions: Array[Partition]) - extends Partition { - override val index = idx - override def hashCode(): Int = idx -} - -private[streaming] -class PartitionAwareUnionRDD[T: ClassManifest]( - sc: SparkContext, - var rdds: Seq[RDD[T]]) - extends RDD[T](sc, rdds.map(x => new OneToOneDependency(x))) { - require(rdds.length > 0) - require(rdds.flatMap(_.partitioner).distinct.length == 1, "Parent RDDs have different partitioners") - - override val partitioner = rdds.head.partitioner - - override def getPartitions: Array[Partition] = { - val numPartitions = rdds.head.partitions.length - (0 until numPartitions).map(index => { - val parentPartitions = rdds.map(_.partitions(index)).toArray - new PartitionAwareUnionRDDPartition(index, parentPartitions) - }).toArray - } - - override def compute(s: Partition, context: TaskContext): Iterator[T] = { - val parentPartitions = s.asInstanceOf[PartitionAwareUnionRDDPartition].partitions - rdds.zip(parentPartitions).iterator.flatMap { - case (rdd, p) => rdd.iterator(p, context) - } - } -} - - - - -- cgit v1.2.3 From fc78f67da2fd28744e8119e28f4bb8a29926b3ad Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Thu, 21 Nov 2013 16:54:23 -0800 Subject: Added logging of scheduler delays to UI --- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 33 ++++++++++++++++++++-- 1 file changed, 31 insertions(+), 2 deletions(-) (limited to 'core/src') 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 fbd822867f..fc8c334cb5 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 @@ -60,11 +60,13 @@ private[spark] class StagePage(parent: JobProgressUI) { var activeTime = 0L listener.stageIdToTasksActive(stageId).foreach(activeTime += _.timeRunning(now)) + val finishedTasks = listener.stageIdToTaskInfos(stageId).filter(_._1.finished) + val summary =
  • - CPU time: + Total duration across all tasks: {parent.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)}
  • {if (hasShuffleRead) @@ -104,6 +106,30 @@ private[spark] class StagePage(parent: JobProgressUI) { val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map( ms => parent.formatDuration(ms.toLong)) + val gettingResultTimes = validTasks.map{case (info, metrics, exception) => + if (info.gettingResultTime > 0) { + (info.finishTime - info.gettingResultTime).toDouble + } else { + 0.0 + } + } + val gettingResultQuantiles = ("Time spent fetching task results" +: + Distribution(gettingResultTimes).get.getQuantiles().map( + millis => parent.formatDuration(millis.toLong))) + // The scheduler delay includes the network delay to send the task to the worker + // machine and to send back the result (but not the time to fetch the task result, + // if it needed to be fetched from the block manager on the worker). + val schedulerDelays = validTasks.map{case (info, metrics, exception) => + if (info.gettingResultTime > 0) { + (info.gettingResultTime - info.launchTime).toDouble + } else { + (info.finishTime - info.launchTime).toDouble + } + } + val schedulerDelayQuantiles = ("Scheduler delay" +: + Distribution(schedulerDelays).get.getQuantiles().map( + millis => parent.formatDuration(millis.toLong))) + def getQuantileCols(data: Seq[Double]) = Distribution(data).get.getQuantiles().map(d => Utils.bytesToString(d.toLong)) @@ -119,7 +145,10 @@ private[spark] class StagePage(parent: JobProgressUI) { } val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes) - val listings: Seq[Seq[String]] = Seq(serviceQuantiles, + val listings: Seq[Seq[String]] = Seq( + serviceQuantiles, + gettingResultQuantiles, + schedulerDelayQuantiles, if (hasShuffleRead) shuffleReadQuantiles else Nil, if (hasShuffleWrite) shuffleWriteQuantiles else Nil) -- cgit v1.2.3 From ab3cefde5349d0de85b23b49feef493ff0b2d1ed Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Wed, 23 Oct 2013 09:42:25 +0800 Subject: Add YarnClientClusterScheduler and Backend. With this scheduler, the user application is launched locally, While the executor will be launched by YARN on remote nodes. This enables spark-shell to run upon YARN. --- .../main/scala/org/apache/spark/SparkContext.scala | 25 +++ docs/running-on-yarn.md | 27 ++- .../org/apache/spark/deploy/yarn/Client.scala | 13 +- .../apache/spark/deploy/yarn/ClientArguments.scala | 40 ++-- .../apache/spark/deploy/yarn/WorkerLauncher.scala | 246 +++++++++++++++++++++ .../cluster/YarnClientClusterScheduler.scala | 47 ++++ .../cluster/YarnClientSchedulerBackend.scala | 109 +++++++++ 7 files changed, 484 insertions(+), 23 deletions(-) create mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala create mode 100644 yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala create mode 100644 yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 42b2985b50..3a80241daa 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -226,6 +226,31 @@ class SparkContext( scheduler.initialize(backend) scheduler + case "yarn-client" => + val scheduler = try { + val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") + val cons = clazz.getConstructor(classOf[SparkContext]) + cons.newInstance(this).asInstanceOf[ClusterScheduler] + + } catch { + case th: Throwable => { + throw new SparkException("YARN mode not available ?", th) + } + } + + val backend = try { + val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend") + val cons = clazz.getConstructor(classOf[ClusterScheduler], classOf[SparkContext]) + cons.newInstance(scheduler, this).asInstanceOf[CoarseGrainedSchedulerBackend] + } catch { + case th: Throwable => { + throw new SparkException("YARN mode not available ?", th) + } + } + + scheduler.initialize(backend) + scheduler + case MESOS_REGEX(mesosUrl) => MesosNativeLibrary.load() val scheduler = new ClusterScheduler(this) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 4056e9c15d..68fd6c2ab1 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -45,6 +45,10 @@ System Properties: Ensure that HADOOP_CONF_DIR or YARN_CONF_DIR points to the directory which contains the (client side) configuration files for the hadoop cluster. This would be used to connect to the cluster, write to the dfs and submit jobs to the resource manager. +There are two scheduler mode that can be used to launch spark application on YARN. + +## Launch spark application by YARN Client with yarn-standalone mode. + The command to launch the YARN Client is as follows: SPARK_JAR= ./spark-class org.apache.spark.deploy.yarn.Client \ @@ -52,6 +56,7 @@ The command to launch the YARN Client is as follows: --class \ --args \ --num-workers \ + --master-class --master-memory \ --worker-memory \ --worker-cores \ @@ -85,11 +90,29 @@ For example: $ cat $YARN_APP_LOGS_DIR/$YARN_APP_ID/container*_000001/stdout Pi is roughly 3.13794 -The above starts a YARN Client programs which periodically polls the Application Master for status updates and displays them in the console. The client will exit once your application has finished running. +The above starts a YARN Client programs which start the default Application Master. Then SparkPi will be run as a child thread of Application Master, YARN Client will periodically polls the Application Master for status updates and displays them in the console. The client will exit once your application has finished running. + +With this mode, your application is actually run on the remote machine where the Application Master is run upon. Thus application that involve local interaction will not work well, e.g. spark-shell. + +## Launch spark application with yarn-client mode. + +With yarn-client mode, the application will be launched locally. Just like running application or spark-shell on Local / Mesos / Standalone mode. The launch method is also the similar with them, just make sure that when you need to specify a master url, use "yarn-client" instead. And you also need to export the env value for SPARK_JAR and SPARK_YARN_APP_JAR + +In order to tune worker core/number/memory etc. You need to export SPARK_WORKER_CORES, SPARK_WORKER_MEMORY, SPARK_WORKER_INSTANCES e.g. by ./conf/spark-env.sh + +For example: + + SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ + SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ + ./run-example org.apache.spark.examples.SparkPi yarn-client + + + SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ + SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ + MASTER=yarn-client ./spark-shell # Important Notes -- 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. - 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. - The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt and your application should use the name as appSees.txt to reference it when running on YARN. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 94e353af2e..bb73f6d337 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -54,9 +54,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // staging directory is private! -> rwx-------- val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short) // app files are world-wide readable and owner writable -> rw-r--r-- - val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) + val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) - def run() { + // for client user who want to monitor app status by itself. + def runApp() = { validateArgs() init(yarnConf) @@ -78,7 +79,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName()) submitApp(appContext) - + appId + } + + def run() { + val appId = runApp() monitorApplication(appId) System.exit(0) } @@ -372,7 +377,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val commands = List[String](javaCommand + " -server " + JAVA_OPTS + - " org.apache.spark.deploy.yarn.ApplicationMaster" + + " " + args.amClass + " --class " + args.userClass + " --jar " + args.userJar + userArgsToString(args) + diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 852dbd7dab..b9dbc3fb87 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -35,6 +35,7 @@ class ClientArguments(val args: Array[String]) { var numWorkers = 2 var amQueue = System.getProperty("QUEUE", "default") var amMemory: Int = 512 + var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" // TODO var inputFormatInfo: List[InputFormatInfo] = null @@ -62,18 +63,22 @@ class ClientArguments(val args: Array[String]) { userArgsBuffer += value args = tail - case ("--master-memory") :: MemoryParam(value) :: tail => - amMemory = value + case ("--master-class") :: value :: tail => + amClass = value args = tail - case ("--num-workers") :: IntParam(value) :: tail => - numWorkers = value + case ("--master-memory") :: MemoryParam(value) :: tail => + amMemory = value args = tail case ("--worker-memory") :: MemoryParam(value) :: tail => workerMemory = value args = tail + case ("--num-workers") :: IntParam(value) :: tail => + numWorkers = value + args = tail + case ("--worker-cores") :: IntParam(value) :: tail => workerCores = value args = tail @@ -119,19 +124,20 @@ class ClientArguments(val args: Array[String]) { System.err.println( "Usage: org.apache.spark.deploy.yarn.Client [options] \n" + "Options:\n" + - " --jar JAR_PATH Path to your application's JAR file (required)\n" + - " --class CLASS_NAME Name of your application's main class (required)\n" + - " --args ARGS Arguments to be passed to your application's main class.\n" + - " Mutliple invocations are possible, each will be passed in order.\n" + - " --num-workers NUM Number of workers to start (Default: 2)\n" + - " --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" + - " --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" + - " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" + - " --name NAME The name of your application (Default: Spark)\n" + - " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" + - " --addJars jars Comma separated list of local jars that want SparkContext.addJar to work with.\n" + - " --files files Comma separated list of files to be distributed with the job.\n" + - " --archives archives Comma separated list of archives to be distributed with the job." + " --jar JAR_PATH Path to your application's JAR file (required)\n" + + " --class CLASS_NAME Name of your application's main class (required)\n" + + " --args ARGS Arguments to be passed to your application's main class.\n" + + " Mutliple invocations are possible, each will be passed in order.\n" + + " --num-workers NUM Number of workers to start (Default: 2)\n" + + " --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" + + " --master-class CLASS_NAME Class Name for Master (Default: spark.deploy.yarn.ApplicationMaster)\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" + + " --name NAME The name of your application (Default: Spark)\n" + + " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" + + " --addJars jars Comma separated list of local jars that want SparkContext.addJar to work with.\n" + + " --files files Comma separated list of files to be distributed with the job.\n" + + " --archives archives Comma separated list of archives to be distributed with the job." ) System.exit(exitCode) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala new file mode 100644 index 0000000000..421a83c87a --- /dev/null +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.yarn + +import java.net.Socket +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.ipc.YarnRPC +import org.apache.hadoop.yarn.util.{ConverterUtils, Records} +import akka.actor._ +import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} +import akka.remote.RemoteClientShutdown +import akka.actor.Terminated +import akka.remote.RemoteClientDisconnected +import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.scheduler.SplitInfo + +class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) extends Logging { + + def this(args: ApplicationMasterArguments) = this(args, new Configuration()) + + private val rpc: YarnRPC = YarnRPC.create(conf) + private var resourceManager: AMRMProtocol = null + private var appAttemptId: ApplicationAttemptId = null + private var reporterThread: Thread = null + private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + + private var yarnAllocator: YarnAllocationHandler = null + private var driverClosed:Boolean = false + + val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0)._1 + var actor: ActorRef = null + + // This actor just working as a monitor to watch on Driver Actor. + class MonitorActor(driverUrl: String) extends Actor { + + var driver: ActorRef = null + + override def preStart() { + logInfo("Listen to driver: " + driverUrl) + driver = context.actorFor(driverUrl) + driver ! "hello" + context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.watch(driver) // Doesn't work with remote actors, but useful for testing + } + + override def receive = { + case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => + logInfo("Driver terminated or disconnected! Shutting down.") + driverClosed = true + } + } + + def run() { + + appAttemptId = getApplicationAttemptId() + resourceManager = registerWithResourceManager() + val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() + + // Compute number of threads for akka + val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory() + + if (minimumMemory > 0) { + val mem = args.workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD + val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0) + + if (numCore > 0) { + // do not override - hits https://issues.apache.org/jira/browse/HADOOP-8406 + // TODO: Uncomment when hadoop is on a version which has this fixed. + // args.workerCores = numCore + } + } + + waitForSparkMaster() + + // Allocate all containers + allocateWorkers() + + // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout + // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. + + val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) + // must be <= timeoutInterval/ 2. + // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM. + // so atleast 1 minute or timeoutInterval / 10 - whichever is higher. + val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L)) + reporterThread = launchReporterThread(interval) + + // Wait for the reporter thread to Finish. + reporterThread.join() + + finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) + actorSystem.shutdown() + + logInfo("Exited") + System.exit(0) + } + + private def getApplicationAttemptId(): ApplicationAttemptId = { + val envs = System.getenv() + val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) + val containerId = ConverterUtils.toContainerId(containerIdString) + val appAttemptId = containerId.getApplicationAttemptId() + logInfo("ApplicationAttemptId: " + appAttemptId) + return appAttemptId + } + + private def registerWithResourceManager(): AMRMProtocol = { + val rmAddress = NetUtils.createSocketAddr(yarnConf.get( + YarnConfiguration.RM_SCHEDULER_ADDRESS, + YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS)) + logInfo("Connecting to ResourceManager at " + rmAddress) + return rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] + } + + private def registerApplicationMaster(): RegisterApplicationMasterResponse = { + logInfo("Registering the ApplicationMaster") + val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest]) + .asInstanceOf[RegisterApplicationMasterRequest] + appMasterRequest.setApplicationAttemptId(appAttemptId) + // Setting this to master host,port - so that the ApplicationReport at client has some sensible info. + // Users can then monitor stderr/stdout on that node if required. + appMasterRequest.setHost(Utils.localHostName()) + appMasterRequest.setRpcPort(0) + // What do we provide here ? Might make sense to expose something sensible later ? + appMasterRequest.setTrackingUrl("") + return resourceManager.registerApplicationMaster(appMasterRequest) + } + + private def waitForSparkMaster() { + logInfo("Waiting for spark driver to be reachable.") + var driverUp = false + val hostport = args.userArgs(0) + val (driverHost, driverPort) = Utils.parseHostPort(hostport) + while(!driverUp) { + try { + val socket = new Socket(driverHost, driverPort) + socket.close() + logInfo("Master now available: " + driverHost + ":" + driverPort) + driverUp = true + } catch { + case e: Exception => + logError("Failed to connect to driver at " + driverHost + ":" + driverPort) + Thread.sleep(100) + } + } + System.setProperty("spark.driver.host", driverHost) + System.setProperty("spark.driver.port", driverPort.toString) + + val driverUrl = "akka://spark@%s:%s/user/%s".format( + driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) + + actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") + } + + + private def allocateWorkers() { + + // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now. + val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = scala.collection.immutable.Map() + + yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args, preferredNodeLocationData) + + logInfo("Allocating " + args.numWorkers + " workers.") + // Wait until all containers have finished + // TODO: This is a bit ugly. Can we make it nicer? + // TODO: Handle container failure + while(yarnAllocator.getNumWorkersRunning < args.numWorkers) { + yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) + Thread.sleep(100) + } + + logInfo("All workers have launched.") + + } + + // TODO: We might want to extend this to allocate more containers in case they die ! + private def launchReporterThread(_sleepTime: Long): Thread = { + val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime + + val t = new Thread { + override def run() { + while (!driverClosed) { + val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning + if (missingWorkerCount > 0) { + logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers") + yarnAllocator.allocateContainers(missingWorkerCount) + } + else sendProgress() + Thread.sleep(sleepTime) + } + } + } + // setting to daemon status, though this is usually not a good idea. + t.setDaemon(true) + t.start() + logInfo("Started progress reporter thread - sleep time : " + sleepTime) + return t + } + + private def sendProgress() { + logDebug("Sending progress") + // simulated with an allocate request with no nodes requested ... + yarnAllocator.allocateContainers(0) + } + + def finishApplicationMaster(status: FinalApplicationStatus) { + + logInfo("finish ApplicationMaster with " + status) + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(appAttemptId) + finishReq.setFinishApplicationStatus(status) + resourceManager.finishApplicationMaster(finishReq) + } + +} + + +object WorkerLauncher { + def main(argStrings: Array[String]) { + val args = new ApplicationMasterArguments(argStrings) + new WorkerLauncher(args).run() + } +} diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala new file mode 100644 index 0000000000..63a0449e5a --- /dev/null +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.cluster + +import org.apache.spark._ +import org.apache.hadoop.conf.Configuration +import org.apache.spark.deploy.yarn.YarnAllocationHandler +import org.apache.spark.util.Utils + +/** + * + * This scheduler launch worker through Yarn - by call into Client to launch WorkerLauncher as AM. + */ +private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) { + + def this(sc: SparkContext) = this(sc, new Configuration()) + + // By default, rack is unknown + override def getRackForHost(hostPort: String): Option[String] = { + val host = Utils.parseHostPort(hostPort)._1 + val retval = YarnAllocationHandler.lookupRack(conf, host) + if (retval != null) Some(retval) else None + } + + override def postStartHook() { + + // The yarn application is running, but the worker might not yet ready + // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt + Thread.sleep(2000L) + logInfo("YarnClientClusterScheduler.postStartHook done") + } +} diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala new file mode 100644 index 0000000000..b206780c78 --- /dev/null +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -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. + */ + +package org.apache.spark.scheduler.cluster + +import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} +import org.apache.spark.{SparkException, Logging, SparkContext} +import org.apache.spark.deploy.yarn.{Client, ClientArguments} + +private[spark] class YarnClientSchedulerBackend( + scheduler: ClusterScheduler, + sc: SparkContext) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + with Logging { + + var client: Client = null + var appId: ApplicationId = null + + override def start() { + super.start() + + val defalutWorkerCores = "2" + val defalutWorkerMemory = "512m" + val defaultWorkerNumber = "1" + + val userJar = System.getenv("SPARK_YARN_APP_JAR") + var workerCores = System.getenv("SPARK_WORKER_CORES") + var workerMemory = System.getenv("SPARK_WORKER_MEMORY") + var workerNumber = System.getenv("SPARK_WORKER_INSTANCES") + + if (userJar == null) + throw new SparkException("env SPARK_YARN_APP_JAR is not set") + + if (workerCores == null) + workerCores = defalutWorkerCores + if (workerMemory == null) + workerMemory = defalutWorkerMemory + if (workerNumber == null) + workerNumber = defaultWorkerNumber + + val driverHost = System.getProperty("spark.driver.host") + val driverPort = System.getProperty("spark.driver.port") + val hostport = driverHost + ":" + driverPort + + val argsArray = Array[String]( + "--class", "notused", + "--jar", userJar, + "--args", hostport, + "--worker-memory", workerMemory, + "--worker-cores", workerCores, + "--num-workers", workerNumber, + "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher" + ) + + val args = new ClientArguments(argsArray) + client = new Client(args) + appId = client.runApp() + waitForApp() + } + + def waitForApp() { + + // TODO : need a better way to find out whether the workers are ready or not + // maybe by resource usage report? + while(true) { + val report = client.getApplicationReport(appId) + + logInfo("Application report from ASM: \n" + + "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + + "\t appStartTime: " + report.getStartTime() + "\n" + + "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + ) + + // Ready to go, or already gone. + val state = report.getYarnApplicationState() + if (state == YarnApplicationState.RUNNING) { + return + } else if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.FAILED || + state == YarnApplicationState.KILLED) { + throw new SparkException("Yarn application already ended," + + "might be killed or not able to launch application master.") + } + + Thread.sleep(1000) + } + } + + override def stop() { + super.stop() + client.stop() + logInfo("Stoped") + } + +} -- cgit v1.2.3 From 4f1c3fa5d7e6fe509b1cea550eaa213a185ec964 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Sat, 23 Nov 2013 17:07:19 -0800 Subject: Hadoop 2.2 YARN API migration for `SPARK_HOME/new-yarn` --- .../main/scala/org/apache/spark/SparkContext.scala | 2 +- .../spark/deploy/yarn/ApplicationMaster.scala | 155 +++--- .../org/apache/spark/deploy/yarn/Client.scala | 163 +++--- .../apache/spark/deploy/yarn/ClientArguments.scala | 19 +- .../apache/spark/deploy/yarn/WorkerRunnable.scala | 48 +- .../spark/deploy/yarn/YarnAllocationHandler.scala | 570 +++++++++++---------- 6 files changed, 468 insertions(+), 489 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 42b2985b50..fad54683bc 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -81,7 +81,7 @@ class SparkContext( val sparkHome: String = null, val jars: Seq[String] = Nil, val environment: Map[String, String] = Map(), - // This is used only by yarn for now, but should be relevant to other cluster types (mesos, etc) + // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, etc) // too. This is typically generated from InputFormatInfo.computePreferredLocations .. host, set // of data-local splits on host val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 9c43a7287d..eeeca3ea8a 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -30,8 +30,10 @@ import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.client.api.AMRMClient +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} @@ -45,55 +47,43 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e def this(args: ApplicationMasterArguments) = this(args, new Configuration()) private var rpc: YarnRPC = YarnRPC.create(conf) - private var resourceManager: AMRMProtocol = _ + private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private var appAttemptId: ApplicationAttemptId = _ private var userThread: Thread = _ - private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private val fs = FileSystem.get(yarnConf) private var yarnAllocator: YarnAllocationHandler = _ private var isFinished: Boolean = false private var uiAddress: String = _ - private val maxAppAttempts: Int = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES, - YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) + private val maxAppAttempts: Int = conf.getInt( + YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) private var isLastAMRetry: Boolean = true - // default to numWorkers * 2, with minimum of 3 + private var amClient: AMRMClient[ContainerRequest] = _ + + // Default to numWorkers * 2, with minimum of 3 private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures", math.max(args.numWorkers * 2, 3).toString()).toInt def run() { - // Setup the directories so things go to yarn approved directories rather - // then user specified and /tmp. + // Setup the directories so things go to YARN approved directories rather + // than user specified and /tmp. System.setProperty("spark.local.dir", getLocalDirs()) - // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using. + // Use priority 30 as it's higher then HDFS. It's same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) - + appAttemptId = getApplicationAttemptId() isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts - resourceManager = registerWithResourceManager() + amClient = AMRMClient.createAMRMClient() + amClient.init(yarnConf) + amClient.start() // Workaround until hadoop moves to something which has // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line) - // ignore result. - // This does not, unfortunately, always work reliably ... but alleviates the bug a lot of times - // Hence args.workerCores = numCore disabled above. Any better option? - - // Compute number of threads for akka - //val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory() - //if (minimumMemory > 0) { - // val mem = args.workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD - // val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0) - - // if (numCore > 0) { - // do not override - hits https://issues.apache.org/jira/browse/HADOOP-8406 - // TODO: Uncomment when hadoop is on a version which has this fixed. - // args.workerCores = numCore - // } - //} // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf) ApplicationMaster.register(this) + // Start the user's JAR userThread = startUserClass() @@ -103,12 +93,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e waitForSparkContextInitialized() - // Do this after spark master is up and SparkContext is created so that we can register UI Url + // Do this after Spark master is up and SparkContext is created so that we can register UI Url. val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() - + // Allocate all containers allocateWorkers() - + // Wait for the user class to Finish userThread.join() @@ -132,41 +122,24 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() - val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) + val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name()) val containerId = ConverterUtils.toContainerId(containerIdString) val appAttemptId = containerId.getApplicationAttemptId() logInfo("ApplicationAttemptId: " + appAttemptId) appAttemptId } - private def registerWithResourceManager(): AMRMProtocol = { - val rmAddress = NetUtils.createSocketAddr(yarnConf.get( - YarnConfiguration.RM_SCHEDULER_ADDRESS, - YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS)) - logInfo("Connecting to ResourceManager at " + rmAddress) - rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] - } - private def registerApplicationMaster(): RegisterApplicationMasterResponse = { logInfo("Registering the ApplicationMaster") - val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest]) - .asInstanceOf[RegisterApplicationMasterRequest] - appMasterRequest.setApplicationAttemptId(appAttemptId) - // Setting this to master host,port - so that the ApplicationReport at client has some - // sensible info. - // Users can then monitor stderr/stdout on that node if required. - appMasterRequest.setHost(Utils.localHostName()) - appMasterRequest.setRpcPort(0) - appMasterRequest.setTrackingUrl(uiAddress) - resourceManager.registerApplicationMaster(appMasterRequest) + amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) } private def waitForSparkMaster() { - logInfo("Waiting for spark driver to be reachable.") + logInfo("Waiting for Spark driver to be reachable.") var driverUp = false var tries = 0 val numTries = System.getProperty("spark.yarn.applicationMaster.waitTries", "10").toInt - while(!driverUp && tries < numTries) { + while (!driverUp && tries < numTries) { val driverHost = System.getProperty("spark.driver.host") val driverPort = System.getProperty("spark.driver.port") try { @@ -176,8 +149,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e driverUp = true } catch { case e: Exception => { - logWarning("Failed to connect to driver at %s:%s, retrying ..."). - format(driverHost, driverPort) + logWarning("Failed to connect to driver at %s:%s, retrying ...". + format(driverHost, driverPort)) Thread.sleep(100) tries = tries + 1 } @@ -218,44 +191,44 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e t } - // this need to happen before allocateWorkers + // This need to happen before allocateWorkers() private def waitForSparkContextInitialized() { - logInfo("Waiting for spark context initialization") + logInfo("Waiting for Spark context initialization") try { var sparkContext: SparkContext = null ApplicationMaster.sparkContextRef.synchronized { - var count = 0 + var numTries = 0 val waitTime = 10000L - val numTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt - while (ApplicationMaster.sparkContextRef.get() == null && count < numTries) { - logInfo("Waiting for spark context initialization ... " + count) - count = count + 1 + val maxNumTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt + while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries) { + logInfo("Waiting for Spark context initialization ... " + numTries) + numTries = numTries + 1 ApplicationMaster.sparkContextRef.wait(waitTime) } sparkContext = ApplicationMaster.sparkContextRef.get() - assert(sparkContext != null || count >= numTries) + assert(sparkContext != null || numTries >= maxNumTries) - if (null != sparkContext) { + if (sparkContext != null) { uiAddress = sparkContext.ui.appUIAddress this.yarnAllocator = YarnAllocationHandler.newAllocator( yarnConf, - resourceManager, + amClient, appAttemptId, args, - sparkContext.preferredNodeLocationData) + sparkContext.preferredNodeLocationData) } else { - logWarning("Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d". - format(count * waitTime, numTries)) + logWarning("Unable to retrieve SparkContext inspite of waiting for %d, maxNumTries = %d". + format(numTries * waitTime, maxNumTries)) this.yarnAllocator = YarnAllocationHandler.newAllocator( yarnConf, - resourceManager, + amClient, appAttemptId, args) } } } finally { - // in case of exceptions, etc - ensure that count is atleast ALLOCATOR_LOOP_WAIT_COUNT : - // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks + // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT : + // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks. ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) } } @@ -266,15 +239,14 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Wait until all containers have finished // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure - - // Exists the loop if the user thread exits. + yarnAllocator.addResourceRequests(args.numWorkers) + // Exits the loop if the user thread exits. while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) { if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { finishApplicationMaster(FinalApplicationStatus.FAILED, "max number of worker failures reached") } - yarnAllocator.allocateContainers( - math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) + yarnAllocator.allocateResources() ApplicationMaster.incrementAllocatorLoop(1) Thread.sleep(100) } @@ -287,7 +259,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Launch a progress reporter thread, else the app will get killed after expiration // (def: 10mins) timeout. - // TODO(harvey): Verify the timeout if (userThread.isAlive) { // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses. val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) @@ -313,13 +284,14 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e finishApplicationMaster(FinalApplicationStatus.FAILED, "max number of worker failures reached") } - val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning + val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning - + yarnAllocator.getNumPendingAllocate if (missingWorkerCount > 0) { logInfo("Allocating %d containers to make up for (potentially) lost containers". format(missingWorkerCount)) - yarnAllocator.allocateContainers(missingWorkerCount) + yarnAllocator.addResourceRequests(missingWorkerCount) } - else sendProgress() + sendProgress() Thread.sleep(sleepTime) } } @@ -333,8 +305,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private def sendProgress() { logDebug("Sending progress") - // Simulated with an allocate request with no nodes requested ... - yarnAllocator.allocateContainers(0) + // Simulated with an allocate request with no nodes requested. + yarnAllocator.allocateResources() } /* @@ -361,14 +333,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } logInfo("finishApplicationMaster with " + status) - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(appAttemptId) - finishReq.setFinishApplicationStatus(status) - finishReq.setDiagnostics(diagnostics) - // Set tracking url to empty since we don't have a history server. - finishReq.setTrackingUrl("") - resourceManager.finishApplicationMaster(finishReq) + // Set tracking URL to empty since we don't have a history server. + amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */) } /** @@ -412,6 +378,14 @@ object ApplicationMaster { // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be // optimal as more containers are available. Might need to handle this better. private val ALLOCATOR_LOOP_WAIT_COUNT = 30 + + private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]() + + val sparkContextRef: AtomicReference[SparkContext] = + new AtomicReference[SparkContext](null /* initialValue */) + + val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0) + def incrementAllocatorLoop(by: Int) { val count = yarnAllocatorLoop.getAndAdd(by) if (count >= ALLOCATOR_LOOP_WAIT_COUNT) { @@ -422,16 +396,11 @@ object ApplicationMaster { } } - private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]() - def register(master: ApplicationMaster) { applicationMasters.add(master) } - val sparkContextRef: AtomicReference[SparkContext] = - new AtomicReference[SparkContext](null /* initialValue */) - val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0) - + // TODO(harvey): See whether this should be discarded - it isn't used anywhere atm... def sparkContextInitialized(sc: SparkContext): Boolean = { var modified = false sparkContextRef.synchronized { diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 86310f32d5..ee90086729 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.YarnClientImpl +import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, Records} @@ -45,10 +45,13 @@ import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil +/** + * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The + * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster, + * which will launch a Spark master process and negotiate resources throughout its duration. + */ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging { - def this(args: ClientArguments) = this(new Configuration(), args) - var rpc: YarnRPC = YarnRPC.create(conf) val yarnConf: YarnConfiguration = new YarnConfiguration(conf) val credentials = UserGroupInformation.getCurrentUser().getCredentials() @@ -56,48 +59,68 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl private val distCacheMgr = new ClientDistributedCacheManager() // Staging directory is private! -> rwx-------- - val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short) + val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700: Short) // App files are world-wide readable and owner writable -> rw-r--r-- - val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) + val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644: Short) + + def this(args: ClientArguments) = this(new Configuration(), args) def run() { validateArgs() + // Initialize and start the client service. init(yarnConf) start() + + // Log details about this YARN cluster (e.g, the number of slave machines/NodeManagers). logClusterResourceDetails() - val newApp = super.getNewApplication() - val appId = newApp.getApplicationId() + // Prepare to submit a request to the ResourcManager (specifically its ApplicationsManager (ASM) + // interface). - verifyClusterResources(newApp) - val appContext = createApplicationSubmissionContext(appId) + // Get a new client application. + val newApp = super.createApplication() + val newAppResponse = newApp.getNewApplicationResponse() + val appId = newAppResponse.getApplicationId() + + verifyClusterResources(newAppResponse) + + // Set up resource and environment variables. val appStagingDir = getAppStagingDir(appId) val localResources = prepareLocalResources(appStagingDir) - val env = setupLaunchEnv(localResources, appStagingDir) - val amContainer = createContainerLaunchContext(newApp, localResources, env) + val launchEnv = setupLaunchEnv(localResources, appStagingDir) + val amContainer = createContainerLaunchContext(newAppResponse, localResources, launchEnv) + // Set up an application submission context. + val appContext = newApp.getApplicationSubmissionContext() + appContext.setApplicationName(args.appName) appContext.setQueue(args.amQueue) appContext.setAMContainerSpec(amContainer) - appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName()) - submitApp(appContext) + // Memory for the ApplicationMaster. + val memoryResource = Records.newRecord(classOf[Resource]).asInstanceOf[Resource] + memoryResource.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + appContext.setResource(memoryResource) + // Finally, submit and monitor the application. + submitApp(appContext) monitorApplication(appId) + System.exit(0) } + // TODO(harvey): This could just go in ClientArguments. def validateArgs() = { Map( (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", (args.userJar == null) -> "Error: You must specify a user jar!", (args.userClass == null) -> "Error: You must specify a user class!", (args.numWorkers <= 0) -> "Error: You must specify atleast 1 worker!", - (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> "Error: AM memory size must be + - greater then: " + YarnAllocationHandler.MEMORY_OVERHEAD, - (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> "Error: Worker memory size + - must be greater then: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString - .foreach { case(cond, errStr) => + (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be" + + "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD), + (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size" + + "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString) + ).foreach { case(cond, errStr) => if (cond) { logError(errStr) args.printUsageAndExit(1) @@ -111,17 +134,17 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl def logClusterResourceDetails() { val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics - logInfo("Got Cluster metric info from ASM, numNodeManagers = " + + logInfo("Got Cluster metric info from ApplicationsManager (ASM), number of NodeManagers: " + clusterMetrics.getNumNodeManagers) val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue) - logInfo("""Queue info ... queueName = %s, queueCurrentCapacity = %s, queueMaxCapacity = %s, + logInfo("""Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s, queueApplicationCount = %s, queueChildQueueCount = %s""".format( queueInfo.getQueueName, queueInfo.getCurrentCapacity, queueInfo.getMaximumCapacity, queueInfo.getApplications.size, - queueInfo.getChildQueues.size) + queueInfo.getChildQueues.size)) } def verifyClusterResources(app: GetNewApplicationResponse) = { @@ -130,25 +153,19 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // If we have requested more then the clusters max for a single resource then exit. if (args.workerMemory > maxMem) { - logError("the worker size is to large to run on this cluster " + args.workerMemory) + logError("Required worker memory (%d MB), is above the max threshold (%d MB) of this cluster.". + format(args.workerMemory, maxMem)) System.exit(1) } val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD if (amMem > maxMem) { - logError("AM size is to large to run on this cluster " + amMem) + logError("Required AM memory (%d) is above the max threshold (%d) of this cluster". + format(args.amMemory, maxMem)) System.exit(1) } // We could add checks to make sure the entire cluster has enough resources but that involves - // getting all the node reports and computing ourselves - } - - def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = { - logInfo("Setting up application submission context for ASM") - val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) - appContext.setApplicationId(appId) - appContext.setApplicationName(args.appName) - return appContext + // getting all the node reports and computing ourselves. } /** See if two file systems are the same or not. */ @@ -213,7 +230,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = { logInfo("Preparing Local resources") // Upload Spark and the application JAR to the remote file system if necessary. Add them as - // local resources to the AM. + // local resources to the application master. val fs = FileSystem.get(conf) val delegTokenRenewer = Master.getMasterPrincipal(conf) @@ -230,18 +247,20 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val dstFs = dst.getFileSystem(conf) dstFs.addDelegationTokens(delegTokenRenewer, credentials) } + val localResources = HashMap[String, LocalResource]() FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION)) val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - Map(Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar, - Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF")) - .foreach { case(destName, _localPath) => + Map( + Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar, + Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF") + ).foreach { case(destName, _localPath) => val localPath: String = if (_localPath != null) _localPath.trim() else "" if (! localPath.isEmpty()) { var localURI = new URI(localPath) - // if not specified assume these are in the local filesystem to keep behavior like Hadoop + // If not specified assume these are in the local filesystem to keep behavior like Hadoop if (localURI.getScheme() == null) { localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString) } @@ -252,19 +271,21 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } - // handle any add jars + // Handle jars local to the ApplicationMaster. if ((args.addJars != null) && (!args.addJars.isEmpty())){ args.addJars.split(',').foreach { case file: String => val localURI = new URI(file.trim()) val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) val destPath = copyRemoteFile(dst, localPath, replication) + // Only add the resource to the Spark ApplicationMaster. + val appMasterOnly = true distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, - linkname, statCache, true) + linkname, statCache, appMasterOnly) } } - // handle any distributed cache files + // Handle any distributed cache files if ((args.files != null) && (!args.files.isEmpty())){ args.files.split(',').foreach { case file: String => val localURI = new URI(file.trim()) @@ -276,7 +297,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } - // handle any distributed cache archives + // Handle any distributed cache archives if ((args.archives != null) && (!args.archives.isEmpty())) { args.archives.split(',').foreach { case file:String => val localURI = new URI(file.trim()) @@ -289,7 +310,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } UserGroupInformation.getCurrentUser().addCredentials(credentials) - return localResources + localResources } def setupLaunchEnv( @@ -311,8 +332,9 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // Allow users to specify some environment variables. Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) - // Add each SPARK-* key to the environment. + // Add each SPARK_* key to the environment. System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } + env } @@ -335,33 +357,32 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl amContainer.setLocalResources(localResources) amContainer.setEnvironment(env) - val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory() - - // TODO(harvey): This can probably be a val. - var amMemory = ((args.amMemory / minResMemory) * minResMemory) + - ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - - YarnAllocationHandler.MEMORY_OVERHEAD) + // TODO: Need a replacement for the following code to fix -Xmx? + // val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory() + // var amMemory = ((args.amMemory / minResMemory) * minResMemory) + + // ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - + // YarnAllocationHandler.MEMORY_OVERHEAD) // Extra options for the JVM var JAVA_OPTS = "" - // Add Xmx for am memory - JAVA_OPTS += "-Xmx" + amMemory + "m " + // Add Xmx for AM memory + JAVA_OPTS += "-Xmx" + args.amMemory + "m" - JAVA_OPTS += " -Djava.io.tmpdir=" + - new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " + val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + JAVA_OPTS += " -Djava.io.tmpdir=" + tmpDir - // 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) - // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in - // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset + // TODO: Remove once cpuset version is pushed out. + // The context is, default gc for server class machines ends up using all cores to do gc - + // hence if there are multiple containers in same node, Spark GC affects all other containers' + // performance (which can be that of other Spark containers) + // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in + // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset // of cores on a node. val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") && java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC")) if (useConcurrentAndIncrementalGC) { - // In our expts, using (default) throughput collector has severe perf ramnifications in + // In our expts, using (default) throughput collector has severe perf ramifications in // multi-tenant machines JAVA_OPTS += " -XX:+UseConcMarkSweepGC " JAVA_OPTS += " -XX:+CMSIncrementalMode " @@ -371,7 +392,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } if (env.isDefinedAt("SPARK_JAVA_OPTS")) { - JAVA_OPTS += env("SPARK_JAVA_OPTS") + " " + JAVA_OPTS += " " + env("SPARK_JAVA_OPTS") } // Command for the ApplicationMaster @@ -381,7 +402,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl javaCommand = Environment.JAVA_HOME.$() + "/bin/java" } - val commands = List[String](javaCommand + + val commands = List[String]( + javaCommand + " -server " + JAVA_OPTS + " org.apache.spark.deploy.yarn.ApplicationMaster" + @@ -393,18 +415,14 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl " --num-workers " + args.numWorkers + " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") - logInfo("Command for the ApplicationMaster: " + commands(0)) - amContainer.setCommands(commands) - val capability = Records.newRecord(classOf[Resource]).asInstanceOf[Resource] - // Memory for the ApplicationMaster. - capability.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD) - amContainer.setResource(capability) + logInfo("Command for starting the Spark ApplicationMaster: " + commands(0)) + amContainer.setCommands(commands) // Setup security tokens. val dob = new DataOutputBuffer() credentials.writeTokenStorageToStream(dob) - amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData())) + amContainer.setTokens(ByteBuffer.wrap(dob.getData())) amContainer } @@ -423,7 +441,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl logInfo("Application report from ASM: \n" + "\t application identifier: " + appId.toString() + "\n" + "\t appId: " + appId.getId() + "\n" + - "\t clientToken: " + report.getClientToken() + "\n" + + "\t clientToAMToken: " + report.getClientToAMToken() + "\n" + "\t appDiagnostics: " + report.getDiagnostics() + "\n" + "\t appMasterHost: " + report.getHost() + "\n" + "\t appQueue: " + report.getQueue() + "\n" + @@ -454,12 +472,13 @@ 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 + // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes - + // see Client#setupLaunchEnv(). System.setProperty("SPARK_YARN_MODE", "true") val args = new ClientArguments(argStrings) - new Client(args).run + (new Client(args)).run() } // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 852dbd7dab..6d3c95867e 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -17,12 +17,14 @@ package org.apache.spark.deploy.yarn -import org.apache.spark.util.MemoryParam -import org.apache.spark.util.IntParam -import collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.{ArrayBuffer, HashMap} + import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo} +import org.apache.spark.util.IntParam +import org.apache.spark.util.MemoryParam + -// TODO: Add code and support for ensuring that yarn resource 'asks' are location aware ! +// TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware ! class ClientArguments(val args: Array[String]) { var addJars: String = null var files: String = null @@ -30,14 +32,16 @@ class ClientArguments(val args: Array[String]) { var userJar: String = null var userClass: String = null var userArgs: Seq[String] = Seq[String]() - var workerMemory = 1024 + var workerMemory = 1024 // MB var workerCores = 1 var numWorkers = 2 var amQueue = System.getProperty("QUEUE", "default") - var amMemory: Int = 512 + var amMemory: Int = 512 // MB var appName: String = "Spark" // TODO var inputFormatInfo: List[InputFormatInfo] = null + // TODO(harvey) + var priority = 0 parseArgs(args.toList) @@ -47,8 +51,7 @@ class ClientArguments(val args: Array[String]) { var args = inputArgs - while (! args.isEmpty) { - + while (!args.isEmpty) { args match { case ("--jar") :: value :: tail => userJar = value diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala index 6a90cc51cf..9f5523c4b9 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala @@ -32,10 +32,12 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.client.api.NMClient import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} +import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} import org.apache.spark.Logging @@ -51,12 +53,14 @@ class WorkerRunnable( extends Runnable with Logging { var rpc: YarnRPC = YarnRPC.create(conf) - var cm: ContainerManager = null + var nmClient: NMClient = _ val yarnConf: YarnConfiguration = new YarnConfiguration(conf) def run = { logInfo("Starting Worker Container") - cm = connectToCM + nmClient = NMClient.createNMClient() + nmClient.init(yarnConf) + nmClient.start() startContainer } @@ -66,8 +70,6 @@ class WorkerRunnable( val ctx = Records.newRecord(classOf[ContainerLaunchContext]) .asInstanceOf[ContainerLaunchContext] - ctx.setContainerId(container.getId()) - ctx.setResource(container.getResource()) val localResources = prepareLocalResources ctx.setLocalResources(localResources) @@ -111,12 +113,10 @@ class WorkerRunnable( } */ - ctx.setUser(UserGroupInformation.getCurrentUser().getShortUserName()) - val credentials = UserGroupInformation.getCurrentUser().getCredentials() val dob = new DataOutputBuffer() credentials.writeTokenStorageToStream(dob) - ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) + ctx.setTokens(ByteBuffer.wrap(dob.getData())) var javaCommand = "java" val javaHome = System.getenv("JAVA_HOME") @@ -144,10 +144,7 @@ class WorkerRunnable( ctx.setCommands(commands) // Send the start request to the ContainerManager - val startReq = Records.newRecord(classOf[StartContainerRequest]) - .asInstanceOf[StartContainerRequest] - startReq.setContainerLaunchContext(ctx) - cm.startContainer(startReq) + nmClient.startContainer(container, ctx) } private def setupDistributedCache( @@ -194,7 +191,7 @@ class WorkerRunnable( } logInfo("Prepared Local resources " + localResources) - return localResources + localResources } def prepareEnvironment: HashMap[String, String] = { @@ -206,30 +203,7 @@ class WorkerRunnable( Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } - return env - } - - def connectToCM: ContainerManager = { - val cmHostPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort() - val cmAddress = NetUtils.createSocketAddr(cmHostPortStr) - logInfo("Connecting to ContainerManager at " + cmHostPortStr) - - // Use doAs and remoteUser here so we can add the container token and not pollute the current - // users credentials with all of the individual container tokens - val user = UserGroupInformation.createRemoteUser(container.getId().toString()) - val containerToken = container.getContainerToken() - if (containerToken != null) { - user.addToken(ProtoUtils.convertFromProtoFormat(containerToken, cmAddress)) - } - - val proxy = user - .doAs(new PrivilegedExceptionAction[ContainerManager] { - def run: ContainerManager = { - return rpc.getProxy(classOf[ContainerManager], - cmAddress, conf).asInstanceOf[ContainerManager] - } - }) - proxy + env } } diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 6ce470e8cb..dba0f7640e 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -32,11 +32,13 @@ import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedu import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api.AMRMProtocol -import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId} +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId import org.apache.hadoop.yarn.api.records.{Container, ContainerId, ContainerStatus} import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest} import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse} +import org.apache.hadoop.yarn.client.api.AMRMClient +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.util.{RackResolver, Records} @@ -56,7 +58,7 @@ object AllocationType extends Enumeration ("HOST", "RACK", "ANY") { // more info on how we are requesting for containers. private[yarn] class YarnAllocationHandler( val conf: Configuration, - val resourceManager: AMRMProtocol, + val amClient: AMRMClient[ContainerRequest], val appAttemptId: ApplicationAttemptId, val maxWorkers: Int, val workerMemory: Int, @@ -83,12 +85,17 @@ private[yarn] class YarnAllocationHandler( // Containers to be released in next request to RM private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean] + // Number of container requests that have been sent to, but not yet allocated by the + // ApplicationMaster. + private val numPendingAllocate = new AtomicInteger() private val numWorkersRunning = new AtomicInteger() // Used to generate a unique id per worker private val workerIdCounter = new AtomicInteger() private val lastResponseId = new AtomicInteger() private val numWorkersFailed = new AtomicInteger() + def getNumPendingAllocate: Int = numPendingAllocate.intValue + def getNumWorkersRunning: Int = numWorkersRunning.intValue def getNumWorkersFailed: Int = numWorkersFailed.intValue @@ -97,154 +104,163 @@ private[yarn] class YarnAllocationHandler( container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) } - def allocateContainers(workersToRequest: Int) { - // We need to send the request only once from what I understand ... but for now, not modifying - // this much. + def releaseContainer(container: Container) { + val containerId = container.getId + pendingReleaseContainers.put(containerId, true) + amClient.releaseAssignedContainer(containerId) + } + + def allocateResources() { + // We have already set the container request. Poll the ResourceManager for a response. + // This doubles as a heartbeat if there are no pending container requests. + val progressIndicator = 0.1f + val allocateResponse = amClient.allocate(progressIndicator) - // Keep polling the Resource Manager for containers - val amResp = allocateWorkerResources(workersToRequest).getAMResponse + val allocatedContainers = allocateResponse.getAllocatedContainers() + if (allocatedContainers.size > 0) { + var numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * allocatedContainers.size) - val _allocatedContainers = amResp.getAllocatedContainers() + if (numPendingAllocateNow < 0) { + numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * numPendingAllocateNow) + } - if (_allocatedContainers.size > 0) { logDebug(""" Allocated containers: %d Current worker count: %d - Containers to-be-released: %d - pendingReleaseContainers: %s + Containers released: %s + Containers to-be-released: %s Cluster resources: %s """.format( allocatedContainers.size, numWorkersRunning.get(), releasedContainerList, pendingReleaseContainers, - amResp.getAvailableResources)) + allocateResponse.getAvailableResources)) val hostToContainers = new HashMap[String, ArrayBuffer[Container]]() - // Ignore if not satisfying constraints { - for (container <- _allocatedContainers) { + for (container <- allocatedContainers) { if (isResourceConstraintSatisfied(container)) { - // allocatedContainers += container - + // Add the accepted `container` to the host's list of already accepted, + // allocated containers val host = container.getNodeId.getHost - val containers = hostToContainers.getOrElseUpdate(host, new ArrayBuffer[Container]()) - - containers += container + val containersForHost = hostToContainers.getOrElseUpdate(host, + new ArrayBuffer[Container]()) + containersForHost += container + } else { + // Release container, since it doesn't satisfy resource constraints. + releaseContainer(container) } - // Add all ignored containers to released list - else releasedContainerList.add(container.getId()) } - // Find the appropriate containers to use. Slightly non trivial groupBy ... + // Find the appropriate containers to use. + // TODO: Cleanup this group-by... val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]() val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]() val offRackContainers = new HashMap[String, ArrayBuffer[Container]]() - for (candidateHost <- hostToContainers.keySet) - { + for (candidateHost <- hostToContainers.keySet) { val maxExpectedHostCount = preferredHostToCount.getOrElse(candidateHost, 0) val requiredHostCount = maxExpectedHostCount - allocatedContainersOnHost(candidateHost) - var remainingContainers = hostToContainers.get(candidateHost).getOrElse(null) - assert(remainingContainers != null) + val remainingContainersOpt = hostToContainers.get(candidateHost) + assert(remainingContainersOpt.isDefined) + var remainingContainers = remainingContainersOpt.get - if (requiredHostCount >= remainingContainers.size){ - // Since we got <= required containers, add all to dataLocalContainers + if (requiredHostCount >= remainingContainers.size) { + // Since we have <= required containers, add all remaining containers to + // `dataLocalContainers`. dataLocalContainers.put(candidateHost, remainingContainers) - // all consumed + // There are no more free containers remaining. remainingContainers = null - } - else if (requiredHostCount > 0) { + } else if (requiredHostCount > 0) { // Container list has more containers than we need for data locality. - // Split into two : data local container count of (remainingContainers.size - - // requiredHostCount) and rest as remainingContainer + // Split the list into two: one based on the data local container count, + // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining + // containers. val (dataLocal, remaining) = remainingContainers.splitAt( remainingContainers.size - requiredHostCount) dataLocalContainers.put(candidateHost, dataLocal) - // remainingContainers = remaining - // yarn has nasty habit of allocating a tonne of containers on a host - discourage this : - // add remaining to release list. If we have insufficient containers, next allocation - // cycle will reallocate (but wont treat it as data local) - for (container <- remaining) releasedContainerList.add(container.getId()) + // Invariant: remainingContainers == remaining + + // YARN has a nasty habit of allocating a ton of containers on a host - discourage this. + // Add each container in `remaining` to list of containers to release. If we have an + // insufficient number of containers, then the next allocation cycle will reallocate + // (but won't treat it as data local). + // TODO(harvey): Rephrase this comment some more. + for (container <- remaining) releaseContainer(container) remainingContainers = null } - // Now rack local - if (remainingContainers != null){ + // For rack local containers + if (remainingContainers != null) { val rack = YarnAllocationHandler.lookupRack(conf, candidateHost) - - if (rack != null){ + if (rack != null) { val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) - val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - - rackLocalContainers.get(rack).getOrElse(List()).size - + val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - + rackLocalContainers.getOrElse(rack, List()).size - if (requiredRackCount >= remainingContainers.size){ - // Add all to dataLocalContainers + if (requiredRackCount >= remainingContainers.size) { + // Add all remaining containers to to `dataLocalContainers`. dataLocalContainers.put(rack, remainingContainers) - // All consumed remainingContainers = null - } - else if (requiredRackCount > 0) { - // container list has more containers than we need for data locality. - // Split into two : data local container count of (remainingContainers.size - - // requiredRackCount) and rest as remainingContainer + } else if (requiredRackCount > 0) { + // Container list has more containers that we need for data locality. + // Split the list into two: one based on the data local container count, + // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining + // containers. val (rackLocal, remaining) = remainingContainers.splitAt( remainingContainers.size - requiredRackCount) val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, new ArrayBuffer[Container]()) existingRackLocal ++= rackLocal + remainingContainers = remaining } } } - // If still not consumed, then it is off rack host - add to that list. - if (remainingContainers != null){ + if (remainingContainers != null) { + // Not all containers have been consumed - add them to the list of off-rack containers. offRackContainers.put(candidateHost, remainingContainers) } } - // Now that we have split the containers into various groups, go through them in order : - // first host local, then rack local and then off rack (everything else). - // Note that the list we create below tries to ensure that not all containers end up within a - // host if there are sufficiently large number of hosts/containers. - - val allocatedContainers = new ArrayBuffer[Container](_allocatedContainers.size) - allocatedContainers ++= ClusterScheduler.prioritizeContainers(dataLocalContainers) - allocatedContainers ++= ClusterScheduler.prioritizeContainers(rackLocalContainers) - allocatedContainers ++= ClusterScheduler.prioritizeContainers(offRackContainers) - - // Run each of the allocated containers - for (container <- allocatedContainers) { + // Now that we have split the containers into various groups, go through them in order: + // first host-local, then rack-local, and finally off-rack. + // Note that the list we create below tries to ensure that not all containers end up within + // a host if there is a sufficiently large number of hosts/containers. + val allocatedContainersToProcess = new ArrayBuffer[Container](allocatedContainers.size) + allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(dataLocalContainers) + allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(rackLocalContainers) + allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(offRackContainers) + + // Run each of the allocated containers. + for (container <- allocatedContainersToProcess) { val numWorkersRunningNow = numWorkersRunning.incrementAndGet() val workerHostname = container.getNodeId.getHost val containerId = container.getId - assert( - container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) + val workerMemoryOverhead = (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + assert(container.getResource.getMemory >= workerMemoryOverhead) if (numWorkersRunningNow > maxWorkers) { - logInfo("""Ignoring container %d at host %s, since we already have the required number of + logInfo("""Ignoring container %s at host %s, since we already have the required number of containers for it.""".format(containerId, workerHostname)) - releasedContainerList.add(containerId) - // reset counter back to old value. + releaseContainer(container) numWorkersRunning.decrementAndGet() - } - else { - // Deallocate + allocate can result in reusing id's wrongly - so use a different counter - // (workerIdCounter) + } else { val workerId = workerIdCounter.incrementAndGet().toString val driverUrl = "akka://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), + System.getProperty("spark.driver.host"), + System.getProperty("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - logInfo("launching container on " + containerId + " host " + workerHostname) - // Just to be safe, simply remove it from pendingReleaseContainers. - // Should not be there, but .. + logInfo("Launching container %s for on host %s".format(containerId, workerHostname)) + + // To be safe, remove the container from `pendingReleaseContainers`. pendingReleaseContainers.remove(containerId) val rack = YarnAllocationHandler.lookupRack(conf, workerHostname) @@ -254,45 +270,52 @@ private[yarn] class YarnAllocationHandler( containerSet += containerId allocatedContainerToHostMap.put(containerId, workerHostname) + if (rack != null) { allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) } } - - new Thread( - new WorkerRunnable(container, conf, driverUrl, workerId, - workerHostname, workerMemory, workerCores) - ).start() + logInfo("Launching WorkerRunnable. driverUrl: %s, workerHostname: %s".format(driverUrl, workerHostname)) + val workerRunnable = new WorkerRunnable( + container, + conf, + driverUrl, + workerId, + workerHostname, + workerMemory, + workerCores) + new Thread(workerRunnable).start() } } logDebug(""" - Finished processing %d completed containers. + Finished allocating %s containers (from %s originally). Current number of workers running: %d, releasedContainerList: %s, pendingReleaseContainers: %s """.format( - completedContainers.size, + allocatedContainersToProcess, + allocatedContainers, numWorkersRunning.get(), releasedContainerList, pendingReleaseContainers)) } + val completedContainers = allocateResponse.getCompletedContainersStatuses() + if (completedContainers.size > 0) { + logDebug("Completed %d containers".format(completedContainers.size)) - val completedContainers = amResp.getCompletedContainersStatuses() - if (completedContainers.size > 0){ - logDebug("Completed %d containers, to-be-released: %s".format( - completedContainers.size, releasedContainerList)) - for (completedContainer <- completedContainers){ + for (completedContainer <- completedContainers) { val containerId = completedContainer.getContainerId - // Was this released by us ? If yes, then simply remove from containerSet and move on. if (pendingReleaseContainers.containsKey(containerId)) { + // YarnAllocationHandler already marked the container for release, so remove it from + // `pendingReleaseContainers`. pendingReleaseContainers.remove(containerId) - } - else { - // Simply decrement count - next iteration of ReporterThread will take care of allocating. + } else { + // Decrement the number of workers running. The next iteration of the ApplicationMaster's + // reporting thread will take care of allocating. numWorkersRunning.decrementAndGet() - logInfo("Completed container %d (state: %s, http address: %s, exit status: %s)".format( + logInfo("Completed container %s (state: %s, exit status: %s)".format( containerId, completedContainer.getState, completedContainer.getExitStatus())) @@ -307,24 +330,32 @@ private[yarn] class YarnAllocationHandler( allocatedHostToContainersMap.synchronized { if (allocatedContainerToHostMap.containsKey(containerId)) { - val host = allocatedContainerToHostMap.get(containerId).getOrElse(null) - assert (host != null) - - val containerSet = allocatedHostToContainersMap.get(host).getOrElse(null) - assert (containerSet != null) - - containerSet -= containerId - if (containerSet.isEmpty) allocatedHostToContainersMap.remove(host) - else allocatedHostToContainersMap.update(host, containerSet) + val hostOpt = allocatedContainerToHostMap.get(containerId) + assert(hostOpt.isDefined) + val host = hostOpt.get + + val containerSetOpt = allocatedHostToContainersMap.get(host) + assert(containerSetOpt.isDefined) + val containerSet = containerSetOpt.get + + containerSet.remove(containerId) + if (containerSet.isEmpty) { + allocatedHostToContainersMap.remove(host) + } else { + allocatedHostToContainersMap.update(host, containerSet) + } - allocatedContainerToHostMap -= containerId + allocatedContainerToHostMap.remove(containerId) - // Doing this within locked context, sigh ... move to outside ? + // TODO: Move this part outside the synchronized block? val rack = YarnAllocationHandler.lookupRack(conf, host) if (rack != null) { val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 - if (rackCount > 0) allocatedRackCount.put(rack, rackCount) - else allocatedRackCount.remove(rack) + if (rackCount > 0) { + allocatedRackCount.put(rack, rackCount) + } else { + allocatedRackCount.remove(rack) + } } } } @@ -342,32 +373,34 @@ private[yarn] class YarnAllocationHandler( } } - def createRackResourceRequests(hostContainers: List[ResourceRequest]): List[ResourceRequest] = { - // First generate modified racks and new set of hosts under it : then issue requests + def createRackResourceRequests( + hostContainers: ArrayBuffer[ContainerRequest] + ): ArrayBuffer[ContainerRequest] = { + // Generate modified racks and new set of hosts under it before issuing requests. val rackToCounts = new HashMap[String, Int]() - // Within this lock - used to read/write to the rack related maps too. for (container <- hostContainers) { - val candidateHost = container.getHostName - val candidateNumContainers = container.getNumContainers + val candidateHost = container.getNodes.last assert(YarnAllocationHandler.ANY_HOST != candidateHost) val rack = YarnAllocationHandler.lookupRack(conf, candidateHost) if (rack != null) { var count = rackToCounts.getOrElse(rack, 0) - count += candidateNumContainers + count += 1 rackToCounts.put(rack, count) } } - val requestedContainers: ArrayBuffer[ResourceRequest] = - new ArrayBuffer[ResourceRequest](rackToCounts.size) - for ((rack, count) <- rackToCounts){ - requestedContainers += - createResourceRequest(AllocationType.RACK, rack, count, YarnAllocationHandler.PRIORITY) + val requestedContainers = new ArrayBuffer[ContainerRequest](rackToCounts.size) + for ((rack, count) <- rackToCounts) { + requestedContainers ++= createResourceRequests( + AllocationType.RACK, + rack, + count, + YarnAllocationHandler.PRIORITY) } - requestedContainers.toList + requestedContainers } def allocatedContainersOnHost(host: String): Int = { @@ -386,147 +419,128 @@ private[yarn] class YarnAllocationHandler( retval } - private def allocateWorkerResources(numWorkers: Int): AllocateResponse = { - - var resourceRequests: List[ResourceRequest] = null - - // default. - if (numWorkers <= 0 || preferredHostToCount.isEmpty) { - logDebug("numWorkers: " + numWorkers + ", host preferences: " + preferredHostToCount.isEmpty) - resourceRequests = List( - createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY)) - } - else { - // request for all hosts in preferred nodes and for numWorkers - - // candidates.size, request by default allocation policy. - val hostContainerRequests: ArrayBuffer[ResourceRequest] = - new ArrayBuffer[ResourceRequest](preferredHostToCount.size) - for ((candidateHost, candidateCount) <- preferredHostToCount) { - val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) - - if (requiredCount > 0) { - hostContainerRequests += createResourceRequest( - AllocationType.HOST, - candidateHost, - requiredCount, - YarnAllocationHandler.PRIORITY) + def addResourceRequests(numWorkers: Int) { + val containerRequests: List[ContainerRequest] = + if (numWorkers <= 0 || preferredHostToCount.isEmpty) { + logDebug("numWorkers: " + numWorkers + ", host preferences: " + + preferredHostToCount.isEmpty) + createResourceRequests( + AllocationType.ANY, + resource = null, + numWorkers, + YarnAllocationHandler.PRIORITY).toList + } else { + // Request for all hosts in preferred nodes and for numWorkers - + // candidates.size, request by default allocation policy. + val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size) + for ((candidateHost, candidateCount) <- preferredHostToCount) { + val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) + + if (requiredCount > 0) { + hostContainerRequests ++= createResourceRequests( + AllocationType.HOST, + candidateHost, + requiredCount, + YarnAllocationHandler.PRIORITY) + } } + val rackContainerRequests: List[ContainerRequest] = createRackResourceRequests( + hostContainerRequests).toList + + val anyContainerRequests = createResourceRequests( + AllocationType.ANY, + resource = null, + numWorkers, + YarnAllocationHandler.PRIORITY) + + val containerRequestBuffer = new ArrayBuffer[ContainerRequest]( + hostContainerRequests.size + rackContainerRequests.size() + anyContainerRequests.size) + + containerRequestBuffer ++= hostContainerRequests + containerRequestBuffer ++= rackContainerRequests + containerRequestBuffer ++= anyContainerRequests + containerRequestBuffer.toList } - val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests( - hostContainerRequests.toList) - val anyContainerRequests: ResourceRequest = createResourceRequest( - AllocationType.ANY, - resource = null, - numWorkers, - YarnAllocationHandler.PRIORITY) - - val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest]( - hostContainerRequests.size() + rackContainerRequests.size() + 1) - - containerRequests ++= hostContainerRequests - containerRequests ++= rackContainerRequests - containerRequests += anyContainerRequests - - resourceRequests = containerRequests.toList + for (request <- containerRequests) { + amClient.addContainerRequest(request) } - val req = Records.newRecord(classOf[AllocateRequest]) - req.setResponseId(lastResponseId.incrementAndGet) - req.setApplicationAttemptId(appAttemptId) - - req.addAllAsks(resourceRequests) - - val releasedContainerList = createReleasedContainerList() - req.addAllReleases(releasedContainerList) - if (numWorkers > 0) { - logInfo("Allocating %d worker containers with %d of memory each.").format(numWorkers, - workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) - } - else { - logDebug("Empty allocation req .. release : " + releasedContainerList) + numPendingAllocate.addAndGet(numWorkers) + logInfo("Will Allocate %d worker containers, each with %d memory".format( + numWorkers, + (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD))) + } else { + logDebug("Empty allocation request ...") } - for (request <- resourceRequests) { - logInfo("ResourceRequest (host : %s, num containers: %d, priority = %d , capability : %s)"). - format( - request.getHostName, - request.getNumContainers, - request.getPriority, - request.getCapability) + for (request <- containerRequests) { + val nodes = request.getNodes + var hostStr = if (nodes == null || nodes.isEmpty) { + "Any" + } else { + nodes.last + } + logInfo("Container request (host: %s, priority: %s, capability: %s".format( + hostStr, + request.getPriority().getPriority, + request.getCapability)) } - resourceManager.allocate(req) } + private def createResourceRequests( + requestType: AllocationType.AllocationType, + resource: String, + numWorkers: Int, + priority: Int + ): ArrayBuffer[ContainerRequest] = { - private def createResourceRequest( - requestType: AllocationType.AllocationType, - resource:String, - numWorkers: Int, - priority: Int): ResourceRequest = { - - // If hostname specified, we need atleast two requests - node local and rack local. - // There must be a third request - which is ANY : that will be specially handled. + // If hostname is specified, then we need at least two requests - node local and rack local. + // There must be a third request, which is ANY. That will be specially handled. requestType match { case AllocationType.HOST => { assert(YarnAllocationHandler.ANY_HOST != resource) val hostname = resource - val nodeLocal = createResourceRequestImpl(hostname, numWorkers, priority) + val nodeLocal = constructContainerRequests( + Array(hostname), + racks = null, + numWorkers, + priority) - // Add to host->rack mapping + // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler. YarnAllocationHandler.populateRackInfo(conf, hostname) - nodeLocal } case AllocationType.RACK => { val rack = resource - createResourceRequestImpl(rack, numWorkers, priority) + constructContainerRequests(hosts = null, Array(rack), numWorkers, priority) } - case AllocationType.ANY => createResourceRequestImpl( - YarnAllocationHandler.ANY_HOST, numWorkers, priority) + case AllocationType.ANY => constructContainerRequests( + hosts = null, racks = null, numWorkers, priority) case _ => throw new IllegalArgumentException( "Unexpected/unsupported request type: " + requestType) } } - private def createResourceRequestImpl( - hostname:String, - numWorkers: Int, - priority: Int): ResourceRequest = { - - val rsrcRequest = Records.newRecord(classOf[ResourceRequest]) - val memCapability = Records.newRecord(classOf[Resource]) - // There probably is some overhead here, let's reserve a bit more memory. - memCapability.setMemory(workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) - rsrcRequest.setCapability(memCapability) + private def constructContainerRequests( + hosts: Array[String], + racks: Array[String], + numWorkers: Int, + priority: Int + ): ArrayBuffer[ContainerRequest] = { - val pri = Records.newRecord(classOf[Priority]) - pri.setPriority(priority) - rsrcRequest.setPriority(pri) + val memoryResource = Records.newRecord(classOf[Resource]) + memoryResource.setMemory(workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) - rsrcRequest.setHostName(hostname) - - rsrcRequest.setNumContainers(java.lang.Math.max(numWorkers, 0)) - rsrcRequest - } + val prioritySetting = Records.newRecord(classOf[Priority]) + prioritySetting.setPriority(priority) - def createReleasedContainerList(): ArrayBuffer[ContainerId] = { - - val retval = new ArrayBuffer[ContainerId](1) - // Iterator on COW list ... - for (container <- releasedContainerList.iterator()){ - retval += container - } - // Remove from the original list. - if (! retval.isEmpty) { - releasedContainerList.removeAll(retval) - for (v <- retval) pendingReleaseContainers.put(v, true) - logInfo("Releasing " + retval.size + " containers. pendingReleaseContainers : " + - pendingReleaseContainers) + val requests = new ArrayBuffer[ContainerRequest]() + for (i <- 0 until numWorkers) { + requests += new ContainerRequest(memoryResource, hosts, racks, prioritySetting) } - - retval + requests } } @@ -537,26 +551,25 @@ object YarnAllocationHandler { // request types (like map/reduce in hadoop for example) val PRIORITY = 1 - // Additional memory overhead - in mb + // Additional memory overhead - in mb. val MEMORY_OVERHEAD = 384 - // Host to rack map - saved from allocation requests - // We are expecting this not to change. - // Note that it is possible for this to change : and RM will indicate that to us via update - // response to allocate. But we are punting on handling that for now. + // Host to rack map - saved from allocation requests. We are expecting this not to change. + // Note that it is possible for this to change : and ResurceManager will indicate that to us via + // update response to allocate. But we are punting on handling that for now. private val hostToRack = new ConcurrentHashMap[String, String]() private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() def newAllocator( - conf: Configuration, - resourceManager: AMRMProtocol, - appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments): YarnAllocationHandler = { - + conf: Configuration, + amClient: AMRMClient[ContainerRequest], + appAttemptId: ApplicationAttemptId, + args: ApplicationMasterArguments + ): YarnAllocationHandler = { new YarnAllocationHandler( conf, - resourceManager, + amClient, appAttemptId, args.numWorkers, args.workerMemory, @@ -566,39 +579,38 @@ object YarnAllocationHandler { } def newAllocator( - conf: Configuration, - resourceManager: AMRMProtocol, - appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments, - map: collection.Map[String, - collection.Set[SplitInfo]]): YarnAllocationHandler = { - - val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) + conf: Configuration, + amClient: AMRMClient[ContainerRequest], + appAttemptId: ApplicationAttemptId, + args: ApplicationMasterArguments, + map: collection.Map[String, + collection.Set[SplitInfo]] + ): YarnAllocationHandler = { + val (hostToSplitCount, rackToSplitCount) = generateNodeToWeight(conf, map) new YarnAllocationHandler( conf, - resourceManager, + amClient, appAttemptId, args.numWorkers, args.workerMemory, args.workerCores, - hostToCount, - rackToCount) + hostToSplitCount, + rackToSplitCount) } def newAllocator( - conf: Configuration, - resourceManager: AMRMProtocol, - appAttemptId: ApplicationAttemptId, - maxWorkers: Int, - workerMemory: Int, - workerCores: Int, - map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = { - + conf: Configuration, + amClient: AMRMClient[ContainerRequest], + appAttemptId: ApplicationAttemptId, + maxWorkers: Int, + workerMemory: Int, + workerCores: Int, + map: collection.Map[String, collection.Set[SplitInfo]] + ): YarnAllocationHandler = { val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) - new YarnAllocationHandler( conf, - resourceManager, + amClient, appAttemptId, maxWorkers, workerMemory, @@ -609,12 +621,13 @@ object YarnAllocationHandler { // A simple method to copy the split info map. private def generateNodeToWeight( - conf: Configuration, - input: collection.Map[String, collection.Set[SplitInfo]]) : - // host to count, rack to count - (Map[String, Int], Map[String, Int]) = { + conf: Configuration, + input: collection.Map[String, collection.Set[SplitInfo]] + ): (Map[String, Int], Map[String, Int]) = { - if (input == null) return (Map[String, Int](), Map[String, Int]()) + if (input == null) { + return (Map[String, Int](), Map[String, Int]()) + } val hostToCount = new HashMap[String, Int] val rackToCount = new HashMap[String, Int] @@ -634,24 +647,25 @@ object YarnAllocationHandler { } def lookupRack(conf: Configuration, host: String): String = { - if (!hostToRack.contains(host)) populateRackInfo(conf, host) + if (!hostToRack.contains(host)) { + populateRackInfo(conf, host) + } hostToRack.get(host) } def fetchCachedHostsForRack(rack: String): Option[Set[String]] = { - val set = rackToHostSet.get(rack) - if (set == null) return None - - // No better way to get a Set[String] from JSet ? - val convertedSet: collection.mutable.Set[String] = set - Some(convertedSet.toSet) + Option(rackToHostSet.get(rack)).map { set => + val convertedSet: collection.mutable.Set[String] = set + // TODO: Better way to get a Set[String] from JSet. + convertedSet.toSet + } } def populateRackInfo(conf: Configuration, hostname: String) { Utils.checkHost(hostname) if (!hostToRack.containsKey(hostname)) { - // If there are repeated failures to resolve, all to an ignore list ? + // If there are repeated failures to resolve, all to an ignore list. val rackInfo = RackResolver.resolve(conf, hostname) if (rackInfo != null && rackInfo.getNetworkLocation != null) { val rack = rackInfo.getNetworkLocation @@ -662,7 +676,7 @@ object YarnAllocationHandler { } rackToHostSet.get(rack).add(hostname) - // TODO(harvey): Figure out this comment... + // TODO(harvey): Figure out what this comment means... // Since RackResolver caches, we are disabling this for now ... } /* else { // right ? Else we will keep calling rack resolver in case we cant resolve rack info ... -- cgit v1.2.3 From e9ff13ec72718ada705b85cc10da1b09bcc86dcc Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 24 Nov 2013 17:56:43 +0800 Subject: Consolidated both mapPartitions related RDDs into a single MapPartitionsRDD. Also changed the semantics of the index parameter in mapPartitionsWithIndex from the partition index of the output partition to the partition index in the current RDD. --- .../org/apache/spark/rdd/MapPartitionsRDD.scala | 10 +++--- .../spark/rdd/MapPartitionsWithContextRDD.scala | 41 ---------------------- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 39 ++++++++++---------- .../scala/org/apache/spark/CheckpointSuite.scala | 2 -- 4 files changed, 22 insertions(+), 70 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala (limited to 'core/src') 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 203179c4ea..ae70d55951 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -20,18 +20,16 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} -private[spark] -class MapPartitionsRDD[U: ClassManifest, T: ClassManifest]( +private[spark] class MapPartitionsRDD[U: ClassManifest, T: ClassManifest]( prev: RDD[T], - f: Iterator[T] => Iterator[U], + f: (TaskContext, Int, Iterator[T]) => Iterator[U], // (TaskContext, partition index, iterator) preservesPartitioning: Boolean = false) extends RDD[U](prev) { - override val partitioner = - if (preservesPartitioning) firstParent[T].partitioner else None + override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None override def getPartitions: Array[Partition] = firstParent[T].partitions override def compute(split: Partition, context: TaskContext) = - f(firstParent[T].iterator(split, context)) + f(context, split.index, firstParent[T].iterator(split, context)) } diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala deleted file mode 100644 index aea08ff81b..0000000000 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala +++ /dev/null @@ -1,41 +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.spark.rdd - -import org.apache.spark.{Partition, TaskContext} - - -/** - * A variant of the MapPartitionsRDD that passes the TaskContext into the closure. From the - * TaskContext, the closure can either get access to the interruptible flag or get the index - * of the partition in the RDD. - */ -private[spark] -class MapPartitionsWithContextRDD[U: ClassManifest, T: ClassManifest]( - prev: RDD[T], - f: (TaskContext, Iterator[T]) => Iterator[U], - preservesPartitioning: Boolean - ) extends RDD[U](prev) { - - override def getPartitions: Array[Partition] = firstParent[T].partitions - - override val partitioner = if (preservesPartitioning) prev.partitioner else None - - override def compute(split: Partition, context: TaskContext) = - f(context, firstParent[T].iterator(split, context)) -} diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 7623c44d88..5b1285307d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -408,7 +408,6 @@ abstract class RDD[T: ClassManifest]( def pipe(command: String, env: Map[String, String]): RDD[String] = new PipedRDD(this, command, env) - /** * Return an RDD created by piping elements to a forked external process. * The print behavior can be customized by providing two functions. @@ -442,7 +441,8 @@ abstract class RDD[T: ClassManifest]( */ def mapPartitions[U: ClassManifest]( f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { - new MapPartitionsRDD(this, sc.clean(f), preservesPartitioning) + val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(iter) + new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning) } /** @@ -451,8 +451,8 @@ abstract class RDD[T: ClassManifest]( */ def mapPartitionsWithIndex[U: ClassManifest]( f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { - val func = (context: TaskContext, iter: Iterator[T]) => f(context.partitionId, iter) - new MapPartitionsWithContextRDD(this, sc.clean(func), preservesPartitioning) + val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter) + new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning) } /** @@ -462,7 +462,8 @@ abstract class RDD[T: ClassManifest]( def mapPartitionsWithContext[U: ClassManifest]( f: (TaskContext, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { - new MapPartitionsWithContextRDD(this, sc.clean(f), preservesPartitioning) + val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(context, iter) + new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning) } /** @@ -483,11 +484,10 @@ abstract class RDD[T: ClassManifest]( def mapWith[A: ClassManifest, U: ClassManifest] (constructA: Int => A, preservesPartitioning: Boolean = false) (f: (T, A) => U): RDD[U] = { - def iterF(context: TaskContext, iter: Iterator[T]): Iterator[U] = { - val a = constructA(context.partitionId) + mapPartitionsWithIndex((index, iter) => { + val a = constructA(index) iter.map(t => f(t, a)) - } - new MapPartitionsWithContextRDD(this, sc.clean(iterF _), preservesPartitioning) + }, preservesPartitioning) } /** @@ -498,11 +498,10 @@ abstract class RDD[T: ClassManifest]( def flatMapWith[A: ClassManifest, U: ClassManifest] (constructA: Int => A, preservesPartitioning: Boolean = false) (f: (T, A) => Seq[U]): RDD[U] = { - def iterF(context: TaskContext, iter: Iterator[T]): Iterator[U] = { - val a = constructA(context.partitionId) + mapPartitionsWithIndex((index, iter) => { + val a = constructA(index) iter.flatMap(t => f(t, a)) - } - new MapPartitionsWithContextRDD(this, sc.clean(iterF _), preservesPartitioning) + }, preservesPartitioning) } /** @@ -511,11 +510,10 @@ abstract class RDD[T: ClassManifest]( * partition with the index of that partition. */ def foreachWith[A: ClassManifest](constructA: Int => A)(f: (T, A) => Unit) { - def iterF(context: TaskContext, iter: Iterator[T]): Iterator[T] = { - val a = constructA(context.partitionId) + mapPartitionsWithIndex { (index, iter) => + val a = constructA(index) iter.map(t => {f(t, a); t}) - } - new MapPartitionsWithContextRDD(this, sc.clean(iterF _), true).foreach(_ => {}) + }.foreach(_ => {}) } /** @@ -524,11 +522,10 @@ abstract class RDD[T: ClassManifest]( * partition with the index of that partition. */ def filterWith[A: ClassManifest](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = { - def iterF(context: TaskContext, iter: Iterator[T]): Iterator[T] = { - val a = constructA(context.partitionId) + mapPartitionsWithIndex((index, iter) => { + val a = constructA(index) iter.filter(t => p(t, a)) - } - new MapPartitionsWithContextRDD(this, sc.clean(iterF _), true) + }, preservesPartitioning = true) } /** diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index f26c44d3e7..d2226aa5a5 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -62,8 +62,6 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { testCheckpointing(_.sample(false, 0.5, 0)) testCheckpointing(_.glom()) testCheckpointing(_.mapPartitions(_.map(_.toString))) - testCheckpointing(r => new MapPartitionsWithContextRDD(r, - (context: TaskContext, iter: Iterator[Int]) => iter.map(_.toString), false )) testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString)) testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x)) testCheckpointing(_.pipe(Seq("cat"))) -- cgit v1.2.3 From 77929cfeed95905106f5b3891e8de1b1c312d119 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 22 Nov 2013 19:46:39 +0530 Subject: Fine tuning defaults for akka and restored tracking of dissassociated events, for they are delivered when a remote TCP socket is closed. Also made transport failure heartbeats larger interval for it is mostly not needed. As we are using remote death watch instead. --- .../org/apache/spark/deploy/master/Master.scala | 23 ++++++++++++++++------ .../org/apache/spark/deploy/worker/Worker.scala | 12 +++++++---- .../executor/CoarseGrainedExecutorBackend.scala | 11 +++++------ .../cluster/CoarseGrainedSchedulerBackend.scala | 3 +++ .../scala/org/apache/spark/util/AkkaUtils.scala | 12 ++++++----- 5 files changed, 40 insertions(+), 21 deletions(-) (limited to 'core/src') 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 a7cfc256a9..25f5927128 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,8 +17,9 @@ package org.apache.spark.deploy.master -import java.util.Date import java.text.SimpleDateFormat +import java.util.concurrent.TimeUnit +import java.util.Date import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await @@ -28,6 +29,7 @@ import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor._ import akka.pattern.ask import akka.remote._ +import akka.serialization.SerializationExtension import akka.util.Timeout import org.apache.spark.{Logging, SparkException} @@ -40,11 +42,6 @@ import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed import org.apache.spark.deploy.DeployMessages.KillExecutor import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged -import scala.Some -import akka.actor.Terminated -import akka.serialization.SerializationExtension -import java.util.concurrent.TimeUnit - private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { import context.dispatcher @@ -102,6 +99,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act override def preStart() { logInfo("Starting Spark master at " + masterUrl) // Listen for remote client disconnection events, since they don't go through Akka's watch() + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) webUi.start() masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort.get context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut) @@ -267,11 +265,20 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act case Terminated(actor) => { // The disconnected actor could've been either a worker or an app; remove whichever of // those we have an entry for in the corresponding actor hashmap + logInfo(s"$actor got terminated, removing it.") actorToWorker.get(actor).foreach(removeWorker) actorToApp.get(actor).foreach(finishApplication) if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() } } + case DisassociatedEvent(_, address, _) => { + // The disconnected client could've been either a worker or an app; remove whichever it was + logInfo(s"$address got disassociated, removing it.") + addressToWorker.get(address).foreach(removeWorker) + addressToApp.get(address).foreach(finishApplication) + if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() } + } + case RequestMasterState => { sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray, state) @@ -431,6 +438,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act exec.id, ExecutorState.LOST, Some("worker lost"), None) exec.application.removeExecutor(exec) } + context.stop(worker.actor) + context.unwatch(worker.actor) persistenceEngine.removeWorker(worker) } @@ -493,6 +502,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act app.driver ! ApplicationRemoved(state.toString) } persistenceEngine.removeApplication(app) + context.stop(app.driver) + context.unwatch(app.driver) schedule() } } 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 9472c9a619..3a7d0b859b 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 @@ -17,14 +17,16 @@ package org.apache.spark.deploy.worker +import java.io.File import java.text.SimpleDateFormat import java.util.Date -import java.io.File import scala.collection.mutable.HashMap import scala.concurrent.duration._ import akka.actor._ +import akka.remote.{ DisassociatedEvent, RemotingLifecycleEvent} + import org.apache.spark.Logging import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ @@ -36,10 +38,8 @@ import org.apache.spark.deploy.DeployMessages.WorkerStateResponse import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed import org.apache.spark.deploy.DeployMessages.KillExecutor import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged -import scala.Some import org.apache.spark.deploy.DeployMessages.Heartbeat import org.apache.spark.deploy.DeployMessages.RegisteredWorker -import akka.remote.DisassociatedEvent import org.apache.spark.deploy.DeployMessages.LaunchExecutor import org.apache.spark.deploy.DeployMessages.RegisterWorker @@ -124,7 +124,7 @@ private[spark] class Worker( logInfo("Spark home: " + sparkHome) createWorkDir() webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) - + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) webUi.start() registerWithMaster() @@ -249,6 +249,10 @@ private[spark] class Worker( logInfo(s"$actor_ terminated !") masterDisconnected() + case x: DisassociatedEvent => + logInfo(s"$x Disassociated !") + masterDisconnected() + case RequestWorkerState => { sender ! WorkerStateResponse(host, port, workerId, executors.values.toList, finishedExecutors.values.toList, activeMasterUrl, cores, memory, diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index a98ec06be9..2818a775d0 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -26,11 +26,6 @@ import org.apache.spark.Logging import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{Utils, AkkaUtils} -import akka.remote.DisassociatedEvent -import akka.remote.AssociationErrorEvent -import akka.remote.DisassociatedEvent -import akka.actor.Terminated - private[spark] class CoarseGrainedExecutorBackend( driverUrl: String, @@ -82,7 +77,11 @@ private[spark] class CoarseGrainedExecutorBackend( } case Terminated(actor) => - logError(s"Driver $actor terminated or disconnected! Shutting down.") + logError(s"Driver $actor terminated, Shutting down.") + System.exit(1) + + case x: DisassociatedEvent => + logError(s"Driver $x disassociated! Shutting down.") System.exit(1) case StopExecutor => diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 821c30a119..e316f6b41f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -121,6 +121,9 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac case Terminated(actor) => actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated")) + case DisassociatedEvent(_, address, _) => + addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated")) + } // Make fake resource offers on all executors diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 2a831382df..90a5387b2b 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -44,9 +44,11 @@ private[spark] object AkkaUtils { val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - val akkaHeartBeatPauses = System.getProperty("spark.akka.pauses", "30").toInt - val akkaFailureDetector = System.getProperty("spark.akka.failure-detector.threshold", "30").toInt - val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "3").toInt + val akkaHeartBeatPauses = System.getProperty("spark.akka.pauses", "60").toInt + val akkaFailureDetector = System.getProperty("spark.akka.failure-detector.threshold", "12.0").toDouble + // Since we have our own Heart Beat mechanism and TCP already tracks connections. + // Using this makes very little sense. So setting this to a relatively larger value suffices. + val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "3").toInt val akkaConf = ConfigFactory.parseString( s""" @@ -56,8 +58,8 @@ private[spark] object AkkaUtils { |akka.remote.watch-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s |akka.remote.watch-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s |akka.remote.watch-failure-detector.threshold = $akkaFailureDetector - |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s - |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s + |akka.remote.transport-failure-detector.heartbeat-interval = 30 s + |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = ${akkaHeartBeatPauses + 10} s |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector |akka.actor.provider = "akka.remote.RemoteActorRefProvider" |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" -- cgit v1.2.3 From 95c55df1c21c1b8a90962415861b27ef91d3b20e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 25 Nov 2013 18:27:06 +0800 Subject: Added unit tests for size estimation for specialized hash sets and maps. --- .../spark/util/collection/OpenHashMapSuite.scala | 16 +++- .../spark/util/collection/OpenHashSetSuite.scala | 20 +++- .../collection/PrimitiveKeyOpenHashMapSuite.scala | 102 +++++++++++++++++++++ .../collection/PrimitiveKeyOpenHashSetSuite.scala | 90 ------------------ 4 files changed, 135 insertions(+), 93 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashSetSuite.scala (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala index ca3f684668..63e874fed3 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala @@ -2,8 +2,20 @@ package org.apache.spark.util.collection import scala.collection.mutable.HashSet import org.scalatest.FunSuite - -class OpenHashMapSuite extends FunSuite { +import org.scalatest.matchers.ShouldMatchers +import org.apache.spark.util.SizeEstimator + +class OpenHashMapSuite extends FunSuite with ShouldMatchers { + + test("size for specialized, primitive value (int)") { + val capacity = 1024 + val map = new OpenHashMap[String, Int](capacity) + val actualSize = SizeEstimator.estimate(map) + // 64 bit for pointers, 32 bit for ints, and 1 bit for the bitset. + val expectedSize = capacity * (64 + 32 + 1) / 8 + // Make sure we are not allocating a significant amount of memory beyond our expected. + actualSize should be <= (expectedSize * 1.1).toLong + } test("initialization") { val goodMap1 = new OpenHashMap[String, Int](1) diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala index 4e11e8a628..4768a1e60b 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala @@ -1,9 +1,27 @@ package org.apache.spark.util.collection import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers +import org.apache.spark.util.SizeEstimator -class OpenHashSetSuite extends FunSuite { + +class OpenHashSetSuite extends FunSuite with ShouldMatchers { + + test("size for specialized, primitive int") { + val loadFactor = 0.7 + val set = new OpenHashSet[Int](64, loadFactor) + for (i <- 0 until 1024) { + set.add(i) + } + assert(set.size === 1024) + assert(set.capacity > 1024) + val actualSize = SizeEstimator.estimate(set) + // 32 bits for the ints + 1 bit for the bitset + val expectedSize = set.capacity * (32 + 1) / 8 + // Make sure we are not allocating a significant amount of memory beyond our expected. + actualSize should be <= (expectedSize * 1.1).toLong + } test("primitive int") { val set = new OpenHashSet[Int] diff --git a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala new file mode 100644 index 0000000000..2220b4f0d5 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala @@ -0,0 +1,102 @@ +package org.apache.spark.util.collection + +import scala.collection.mutable.HashSet +import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers +import org.apache.spark.util.SizeEstimator + +class PrimitiveKeyOpenHashMapSuite extends FunSuite with ShouldMatchers { + + test("size for specialized, primitive key, value (int, int)") { + val capacity = 1024 + val map = new PrimitiveKeyOpenHashMap[Int, Int](capacity) + val actualSize = SizeEstimator.estimate(map) + // 32 bit for keys, 32 bit for values, and 1 bit for the bitset. + val expectedSize = capacity * (32 + 32 + 1) / 8 + // Make sure we are not allocating a significant amount of memory beyond our expected. + actualSize should be <= (expectedSize * 1.1).toLong + } + + test("initialization") { + val goodMap1 = new PrimitiveKeyOpenHashMap[Int, Int](1) + assert(goodMap1.size === 0) + val goodMap2 = new PrimitiveKeyOpenHashMap[Int, Int](255) + assert(goodMap2.size === 0) + val goodMap3 = new PrimitiveKeyOpenHashMap[Int, Int](256) + assert(goodMap3.size === 0) + intercept[IllegalArgumentException] { + new PrimitiveKeyOpenHashMap[Int, Int](1 << 30) // Invalid map size: bigger than 2^29 + } + intercept[IllegalArgumentException] { + new PrimitiveKeyOpenHashMap[Int, Int](-1) + } + intercept[IllegalArgumentException] { + new PrimitiveKeyOpenHashMap[Int, Int](0) + } + } + + test("basic operations") { + val longBase = 1000000L + val map = new PrimitiveKeyOpenHashMap[Long, Int] + + for (i <- 1 to 1000) { + map(i + longBase) = i + assert(map(i + longBase) === i) + } + + assert(map.size === 1000) + + for (i <- 1 to 1000) { + assert(map(i + longBase) === i) + } + + // Test iterator + val set = new HashSet[(Long, Int)] + for ((k, v) <- map) { + set.add((k, v)) + } + assert(set === (1 to 1000).map(x => (x + longBase, x)).toSet) + } + + test("null values") { + val map = new PrimitiveKeyOpenHashMap[Long, String]() + for (i <- 1 to 100) { + map(i.toLong) = null + } + assert(map.size === 100) + assert(map(1.toLong) === null) + } + + test("changeValue") { + val map = new PrimitiveKeyOpenHashMap[Long, String]() + for (i <- 1 to 100) { + map(i.toLong) = i.toString + } + assert(map.size === 100) + for (i <- 1 to 100) { + val res = map.changeValue(i.toLong, { assert(false); "" }, v => { + assert(v === i.toString) + v + "!" + }) + assert(res === i + "!") + } + // Iterate from 101 to 400 to make sure the map grows a couple of times, because we had a + // bug where changeValue would return the wrong result when the map grew on that insert + for (i <- 101 to 400) { + val res = map.changeValue(i.toLong, { i + "!" }, v => { assert(false); v }) + assert(res === i + "!") + } + assert(map.size === 400) + } + + test("inserting in capacity-1 map") { + val map = new PrimitiveKeyOpenHashMap[Long, String](1) + for (i <- 1 to 100) { + map(i.toLong) = i.toString + } + assert(map.size === 100) + for (i <- 1 to 100) { + assert(map(i.toLong) === i.toString) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashSetSuite.scala deleted file mode 100644 index dfd6aed2c4..0000000000 --- a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashSetSuite.scala +++ /dev/null @@ -1,90 +0,0 @@ -package org.apache.spark.util.collection - -import scala.collection.mutable.HashSet -import org.scalatest.FunSuite - -class PrimitiveKeyOpenHashSetSuite extends FunSuite { - - test("initialization") { - val goodMap1 = new PrimitiveKeyOpenHashMap[Int, Int](1) - assert(goodMap1.size === 0) - val goodMap2 = new PrimitiveKeyOpenHashMap[Int, Int](255) - assert(goodMap2.size === 0) - val goodMap3 = new PrimitiveKeyOpenHashMap[Int, Int](256) - assert(goodMap3.size === 0) - intercept[IllegalArgumentException] { - new PrimitiveKeyOpenHashMap[Int, Int](1 << 30) // Invalid map size: bigger than 2^29 - } - intercept[IllegalArgumentException] { - new PrimitiveKeyOpenHashMap[Int, Int](-1) - } - intercept[IllegalArgumentException] { - new PrimitiveKeyOpenHashMap[Int, Int](0) - } - } - - test("basic operations") { - val longBase = 1000000L - val map = new PrimitiveKeyOpenHashMap[Long, Int] - - for (i <- 1 to 1000) { - map(i + longBase) = i - assert(map(i + longBase) === i) - } - - assert(map.size === 1000) - - for (i <- 1 to 1000) { - assert(map(i + longBase) === i) - } - - // Test iterator - val set = new HashSet[(Long, Int)] - for ((k, v) <- map) { - set.add((k, v)) - } - assert(set === (1 to 1000).map(x => (x + longBase, x)).toSet) - } - - test("null values") { - val map = new PrimitiveKeyOpenHashMap[Long, String]() - for (i <- 1 to 100) { - map(i.toLong) = null - } - assert(map.size === 100) - assert(map(1.toLong) === null) - } - - test("changeValue") { - val map = new PrimitiveKeyOpenHashMap[Long, String]() - for (i <- 1 to 100) { - map(i.toLong) = i.toString - } - assert(map.size === 100) - for (i <- 1 to 100) { - val res = map.changeValue(i.toLong, { assert(false); "" }, v => { - assert(v === i.toString) - v + "!" - }) - assert(res === i + "!") - } - // Iterate from 101 to 400 to make sure the map grows a couple of times, because we had a - // bug where changeValue would return the wrong result when the map grew on that insert - for (i <- 101 to 400) { - val res = map.changeValue(i.toLong, { i + "!" }, v => { assert(false); v }) - assert(res === i + "!") - } - assert(map.size === 400) - } - - test("inserting in capacity-1 map") { - val map = new PrimitiveKeyOpenHashMap[Long, String](1) - for (i <- 1 to 100) { - map(i.toLong) = i.toString - } - assert(map.size === 100) - for (i <- 1 to 100) { - assert(map(i.toLong) === i.toString) - } - } -} -- cgit v1.2.3 From 466fd06475d8ed262c456421ed2dceba54229db1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 25 Nov 2013 18:27:26 +0800 Subject: Incorporated ideas from pull request #200. - Use Murmur Hash 3 finalization step to scramble the bits of HashCode instead of the simpler version in java.util.HashMap; the latter one had trouble with ranges of consecutive integers. Murmur Hash 3 is used by fastutil. - Don't check keys for equality when re-inserting due to growing the table; the keys will already be unique - Remember the grow threshold instead of recomputing it on each insert --- .../apache/spark/util/collection/OpenHashSet.scala | 107 +++++++++++---------- 1 file changed, 57 insertions(+), 50 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index 4592e4f939..40986e3731 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -79,6 +79,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest]( protected var _capacity = nextPowerOf2(initialCapacity) protected var _mask = _capacity - 1 protected var _size = 0 + protected var _growThreshold = (loadFactor * _capacity).toInt protected var _bitset = new BitSet(_capacity) @@ -115,7 +116,29 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest]( * @return The position where the key is placed, plus the highest order bit is set if the key * exists previously. */ - def addWithoutResize(k: T): Int = putInto(_bitset, _data, k) + def addWithoutResize(k: T): Int = { + var pos = hashcode(hasher.hash(k)) & _mask + var i = 1 + while (true) { + if (!_bitset.get(pos)) { + // This is a new key. + _data(pos) = k + _bitset.set(pos) + _size += 1 + return pos | NONEXISTENCE_MASK + } else if (_data(pos) == k) { + // Found an existing key. + return pos + } else { + val delta = i + pos = (pos + delta) & _mask + i += 1 + } + } + // Never reached here + assert(INVALID_POS != INVALID_POS) + INVALID_POS + } /** * Rehash the set if it is overloaded. @@ -126,7 +149,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest]( * to a new position (in the new data array). */ def rehashIfNeeded(k: T, allocateFunc: (Int) => Unit, moveFunc: (Int, Int) => Unit) { - if (_size > loadFactor * _capacity) { + if (_size > _growThreshold) { rehash(k, allocateFunc, moveFunc) } } @@ -160,37 +183,6 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest]( */ def nextPos(fromPos: Int): Int = _bitset.nextSetBit(fromPos) - /** - * Put an entry into the set. Return the position where the key is placed. In addition, the - * highest bit in the returned position is set if the key exists prior to this put. - * - * This function assumes the data array has at least one empty slot. - */ - private def putInto(bitset: BitSet, data: Array[T], k: T): Int = { - val mask = data.length - 1 - var pos = hashcode(hasher.hash(k)) & mask - var i = 1 - while (true) { - if (!bitset.get(pos)) { - // This is a new key. - data(pos) = k - bitset.set(pos) - _size += 1 - return pos | NONEXISTENCE_MASK - } else if (data(pos) == k) { - // Found an existing key. - return pos - } else { - val delta = i - pos = (pos + delta) & mask - i += 1 - } - } - // Never reached here - assert(INVALID_POS != INVALID_POS) - INVALID_POS - } - /** * Double the table's size and re-hash everything. We are not really using k, but it is declared * so Scala compiler can specialize this method (which leads to calling the specialized version @@ -204,34 +196,49 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest]( */ private def rehash(k: T, allocateFunc: (Int) => Unit, moveFunc: (Int, Int) => Unit) { val newCapacity = _capacity * 2 - require(newCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") - allocateFunc(newCapacity) - val newData = new Array[T](newCapacity) val newBitset = new BitSet(newCapacity) - var pos = 0 - _size = 0 - while (pos < _capacity) { - if (_bitset.get(pos)) { - val newPos = putInto(newBitset, newData, _data(pos)) - moveFunc(pos, newPos & POSITION_MASK) + val newData = new Array[T](newCapacity) + val newMask = newCapacity - 1 + + var oldPos = 0 + while (oldPos < capacity) { + if (_bitset.get(oldPos)) { + val key = _data(oldPos) + var newPos = hashcode(hasher.hash(key)) & newMask + var i = 1 + var keepGoing = true + // No need to check for equality here when we insert so this has one less if branch than + // the similar code path in addWithoutResize. + while (keepGoing) { + if (!newBitset.get(newPos)) { + // Inserting the key at newPos + newData(newPos) = key + newBitset.set(newPos) + moveFunc(oldPos, newPos) + keepGoing = false + } else { + val delta = i + newPos = (newPos + delta) & newMask + i += 1 + } + } } - pos += 1 + oldPos += 1 } + _bitset = newBitset _data = newData _capacity = newCapacity - _mask = newCapacity - 1 + _mask = newMask + _growThreshold = (loadFactor * newCapacity).toInt } /** - * Re-hash a value to deal better with hash functions that don't differ - * in the lower bits, similar to java.util.HashMap + * Re-hash a value to deal better with hash functions that don't differ in the lower bits. + * We use the Murmur Hash 3 finalization step that's also used in fastutil. */ - private def hashcode(h: Int): Int = { - val r = h ^ (h >>> 20) ^ (h >>> 12) - r ^ (r >>> 7) ^ (r >>> 4) - } + private def hashcode(h: Int): Int = it.unimi.dsi.fastutil.HashCommon.murmurHash3(h) private def nextPowerOf2(n: Int): Int = { val highBit = Integer.highestOneBit(n) -- cgit v1.2.3 From 489862a65766d30278c186d280c6286937c81155 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 25 Nov 2013 17:55:21 +0530 Subject: Remote death watch has a funny bug. https://gist.github.com/ScrapCodes/4805fd84906e40b7b03d --- .../scala/org/apache/spark/deploy/client/Client.scala | 10 ++++------ .../scala/org/apache/spark/deploy/master/Master.scala | 17 ----------------- .../scala/org/apache/spark/deploy/worker/Worker.scala | 9 --------- .../spark/executor/CoarseGrainedExecutorBackend.scala | 5 ----- .../cluster/CoarseGrainedSchedulerBackend.scala | 4 ---- .../main/scala/org/apache/spark/util/AkkaUtils.scala | 11 +++-------- spark-class | 1 + 7 files changed, 8 insertions(+), 49 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 070f10f729..408692ec9c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -23,7 +23,6 @@ import scala.concurrent.duration._ import scala.concurrent.Await import akka.actor._ -import akka.actor.Terminated import akka.pattern.AskTimeoutException import akka.pattern.ask import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent, AssociationErrorEvent} @@ -62,6 +61,7 @@ private[spark] class Client( var alreadyDead = false // To avoid calling listener.dead() multiple times override def preStart() { + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) try { registerWithMaster() } catch { @@ -107,7 +107,6 @@ private[spark] class Client( override def receive = { case RegisteredApplication(appId_, masterUrl) => - context.watch(sender) prevMaster = sender appId = appId_ registered = true @@ -123,7 +122,7 @@ private[spark] class Client( val fullId = appId + "/" + id logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, cores)) listener.executorAdded(fullId, workerId, hostPort, cores, memory) - + case ExecutorUpdated(id, state, message, exitStatus) => val fullId = appId + "/" + id val messageText = message.map(s => " (" + s + ")").getOrElse("") @@ -134,13 +133,12 @@ private[spark] class Client( case MasterChanged(masterUrl, masterWebUiUrl) => logInfo("Master has changed, new master is at " + masterUrl) - context.unwatch(prevMaster) changeMaster(masterUrl) alreadyDisconnected = false sender ! MasterChangeAcknowledged(appId) - case Terminated(actor_) => - logWarning(s"Connection to $actor_ failed; waiting for master to reconnect...") + case DisassociatedEvent(_, address, _) => + logWarning(s"Connection to $address failed; waiting for master to reconnect...") markDisconnected() case StopClient => 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 25f5927128..81fb5c4e43 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 @@ -147,9 +147,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act RecoveryState.ALIVE else RecoveryState.RECOVERING - logInfo("I have been elected leader! New state: " + state) - if (state == RecoveryState.RECOVERING) { beginRecovery(storedApps, storedWorkers) context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis) { completeRecovery() } @@ -171,7 +169,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } else { val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress) registerWorker(worker) - context.watch(sender) persistenceEngine.addWorker(worker) sender ! RegisteredWorker(masterUrl, masterWebUiUrl) schedule() @@ -186,7 +183,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val app = createApplication(description, sender) registerApplication(app) logInfo("Registered app " + description.name + " with ID " + app.id) - context.watch(sender) persistenceEngine.addApplication(app) sender ! RegisteredApplication(app.id, masterUrl) schedule() @@ -262,15 +258,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act if (canCompleteRecovery) { completeRecovery() } } - case Terminated(actor) => { - // The disconnected actor could've been either a worker or an app; remove whichever of - // those we have an entry for in the corresponding actor hashmap - logInfo(s"$actor got terminated, removing it.") - actorToWorker.get(actor).foreach(removeWorker) - actorToApp.get(actor).foreach(finishApplication) - if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() } - } - case DisassociatedEvent(_, address, _) => { // The disconnected client could've been either a worker or an app; remove whichever it was logInfo(s"$address got disassociated, removing it.") @@ -438,8 +425,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act exec.id, ExecutorState.LOST, Some("worker lost"), None) exec.application.removeExecutor(exec) } - context.stop(worker.actor) - context.unwatch(worker.actor) persistenceEngine.removeWorker(worker) } @@ -502,8 +487,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act app.driver ! ApplicationRemoved(state.toString) } persistenceEngine.removeApplication(app) - context.stop(app.driver) - context.unwatch(app.driver) schedule() } } 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 3a7d0b859b..0a183afd8e 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 @@ -73,7 +73,6 @@ private[spark] class Worker( val masterLock: Object = new Object() var master: ActorSelection = null - var prevMaster: ActorRef = null var activeMasterUrl: String = "" var activeMasterWebUiUrl : String = "" @volatile var registered = false @@ -173,8 +172,6 @@ private[spark] class Worker( case RegisteredWorker(masterUrl, masterWebUiUrl) => logInfo("Successfully registered with master " + masterUrl) registered = true - context.watch(sender) // remote death watch for master - prevMaster = sender changeMaster(masterUrl, masterWebUiUrl) context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis, self, SendHeartbeat) @@ -185,8 +182,6 @@ private[spark] class Worker( case MasterChanged(masterUrl, masterWebUiUrl) => logInfo("Master has changed, new master is at " + masterUrl) - context.unwatch(prevMaster) - prevMaster = sender changeMaster(masterUrl, masterWebUiUrl) val execs = executors.values. @@ -245,10 +240,6 @@ private[spark] class Worker( } } - case Terminated(actor_) => - logInfo(s"$actor_ terminated !") - masterDisconnected() - case x: DisassociatedEvent => logInfo(s"$x Disassociated !") masterDisconnected() diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 2818a775d0..dcb12bed4e 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -51,7 +51,6 @@ private[spark] class CoarseGrainedExecutorBackend( override def receive = { case RegisteredExecutor(sparkProperties) => logInfo("Successfully registered with driver") - context.watch(sender) //Start watching for terminated messages. // Make this host instead of hostPort ? executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties) @@ -76,10 +75,6 @@ private[spark] class CoarseGrainedExecutorBackend( executor.killTask(taskId) } - case Terminated(actor) => - logError(s"Driver $actor terminated, Shutting down.") - System.exit(1) - case x: DisassociatedEvent => logError(s"Driver $x disassociated! Shutting down.") System.exit(1) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index e316f6b41f..d614dcbdd8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -73,7 +73,6 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac } else { logInfo("Registered executor: " + sender + " with ID " + executorId) sender ! RegisteredExecutor(sparkProperties) - context.watch(sender) executorActor(executorId) = sender executorHost(executorId) = Utils.parseHostPort(hostPort)._1 freeCores(executorId) = cores @@ -118,9 +117,6 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac removeExecutor(executorId, reason) sender ! true - case Terminated(actor) => - actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated")) - case DisassociatedEvent(_, address, _) => addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated")) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 90a5387b2b..23e9b735f3 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -46,20 +46,15 @@ private[spark] object AkkaUtils { val akkaHeartBeatPauses = System.getProperty("spark.akka.pauses", "60").toInt val akkaFailureDetector = System.getProperty("spark.akka.failure-detector.threshold", "12.0").toDouble - // Since we have our own Heart Beat mechanism and TCP already tracks connections. - // Using this makes very little sense. So setting this to a relatively larger value suffices. - val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "3").toInt + val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "5").toInt val akkaConf = ConfigFactory.parseString( s""" |akka.daemonic = on |akka.loggers = [""akka.event.slf4j.Slf4jLogger""] |akka.stdout-loglevel = "ERROR" - |akka.remote.watch-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s - |akka.remote.watch-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s - |akka.remote.watch-failure-detector.threshold = $akkaFailureDetector - |akka.remote.transport-failure-detector.heartbeat-interval = 30 s - |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = ${akkaHeartBeatPauses + 10} s + |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s + |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector |akka.actor.provider = "akka.remote.RemoteActorRefProvider" |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" diff --git a/spark-class b/spark-class index 78d6e073b1..713404d077 100755 --- a/spark-class +++ b/spark-class @@ -136,3 +136,4 @@ fi exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" + -- cgit v1.2.3 From 7222ee29779c3c5146aa5a3d6d060f3b039c1ff7 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 25 Nov 2013 21:06:42 -0800 Subject: Fix the test --- core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala | 4 ++-- core/src/test/scala/org/apache/spark/JavaAPISuite.java | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) (limited to 'core/src') 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 70f7f01d2b..dad5c72e1c 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 @@ -191,8 +191,8 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav * the maximum value of the last position and all NaN entries will be counted * in that bucket. */ - def histogram(buckets: Array[Double]): Array[Long] = { - srdd.histogram(buckets.map(_.toDouble), false) + def histogram(buckets: Array[scala.Double]): Array[Long] = { + srdd.histogram(buckets, false) } def histogram(buckets: Array[Double], evenBuckets: Boolean): Array[Long] = { diff --git a/core/src/test/scala/org/apache/spark/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java index 8a9c6e63e0..44483fd4ab 100644 --- a/core/src/test/scala/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java @@ -368,10 +368,10 @@ public class JavaAPISuite implements Serializable { public void javaDoubleRDDHistoGram() { JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); // Test using generated buckets - Tuple2 results = rdd.histogram(2); - Double[] expected_buckets = {1.0, 2.5, 4.0}; + Tuple2 results = rdd.histogram(2); + double[] expected_buckets = {1.0, 2.5, 4.0}; long[] expected_counts = {2, 2}; - Assert.assertArrayEquals(expected_buckets, results._1); + Assert.assertArrayEquals(expected_buckets, results._1, 0.1); Assert.assertArrayEquals(expected_counts, results._2); // Test with provided buckets long[] histogram = rdd.histogram(expected_buckets); -- cgit v1.2.3 From 297c09d4bb26ba815c7fcb0a0ff04974959f551e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 25 Nov 2013 22:51:33 -0800 Subject: Improve docs for shuffle instrumentation --- .../org/apache/spark/executor/TaskMetrics.scala | 23 ++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 0b4892f98f..c0ce46e379 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -61,50 +61,53 @@ object TaskMetrics { class ShuffleReadMetrics extends Serializable { /** - * Time when shuffle finishs + * Absolute time when this task finished reading shuffle data */ var shuffleFinishTime: Long = _ /** - * Total number of blocks fetched in a shuffle (remote or local) + * Number of blocks fetched in this shuffle by this task (remote or local) */ var totalBlocksFetched: Int = _ /** - * Number of remote blocks fetched in a shuffle + * Number of remote blocks fetched in this shuffle by this task */ var remoteBlocksFetched: Int = _ /** - * Local blocks fetched in a shuffle + * Number of local blocks fetched in this shuffle by this task */ var localBlocksFetched: Int = _ /** - * Total time that is spent blocked waiting for shuffle to fetch data + * Time the task spent waiting for remote shuffle blocks. This only includes the time + * blocking on shuffle input data. For instance if block B is being fetched while the task is + * still not finished processing block A, it is not considered to be blocking on block B. */ var fetchWaitTime: Long = _ /** - * The total amount of time for all the shuffle fetches. This adds up time from overlapping - * shuffles, so can be longer than task time + * Total time spent fetching remote shuffle blocks. This aggregates the time spent fetching all + * input blocks. Since block fetches are both pipelined and parallelized, this can + * exceed fetchWaitTime and executorRunTime. */ var remoteFetchTime: Long = _ /** - * Total number of remote bytes read from a shuffle + * Total number of remote bytes read from the shuffle by this task */ var remoteBytesRead: Long = _ } class ShuffleWriteMetrics extends Serializable { /** - * Number of bytes written for a shuffle + * Number of bytes written for the shuffle by this task */ var shuffleBytesWritten: Long = _ /** - * Time spent blocking on writes to disk or buffer cache, in nanoseconds. + * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds */ var shuffleWriteTime: Long = _ } -- cgit v1.2.3 From d092a8cc6a75ed4d38cf1c3c065dfe05ca4f695d Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 26 Nov 2013 14:26:23 +0530 Subject: Fixed compile time warnings and formatting post merge. --- .../apache/spark/deploy/master/RecoveryState.scala | 4 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 4 +- .../org/apache/spark/util/MetadataCleaner.scala | 3 +- .../org/apache/spark/storage/BlockIdSuite.scala | 2 +- .../scala/org/apache/spark/repl/ReplSuite.scala | 139 +++++++++++---------- 5 files changed, 79 insertions(+), 73 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala index b91be821f0..256a5a7c28 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala @@ -17,9 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object RecoveryState - extends Enumeration("STANDBY", "ALIVE", "RECOVERING", "COMPLETING_RECOVERY") { - +private[spark] object RecoveryState extends Enumeration { type MasterState = Value val STANDBY, ALIVE, RECOVERING, COMPLETING_RECOVERY = Value diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 5f6407aadc..da18d45e65 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -553,7 +553,7 @@ abstract class RDD[T: ClassTag]( (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, false) - def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest] + def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag] (rdd2: RDD[B], rdd3: RDD[C], preservesPartitioning: Boolean) (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, preservesPartitioning) @@ -563,7 +563,7 @@ abstract class RDD[T: ClassTag]( (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, false) - def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest] + def zipPartitions[B: ClassTag, C: ClassTag, D: ClassTag, V: ClassTag] (rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D], preservesPartitioning: Boolean) (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] = new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4, preservesPartitioning) diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 67a7f87a5c..7b41ef89f1 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -55,8 +55,7 @@ class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, clea } } -object MetadataCleanerType extends Enumeration("MapOutputTracker", "SparkContext", "HttpBroadcast", "DagScheduler", "ResultTask", - "ShuffleMapTask", "BlockManager", "DiskBlockManager", "BroadcastVars") { +object MetadataCleanerType extends Enumeration { val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, DAG_SCHEDULER, RESULT_TASK, SHUFFLE_MAP_TASK, BLOCK_MANAGER, SHUFFLE_BLOCK_MANAGER, BROADCAST_VARS = Value diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala index cb76275e39..b647e8a672 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -39,7 +39,7 @@ class BlockIdSuite extends FunSuite { fail() } catch { case e: IllegalStateException => // OK - case _ => fail() + case _: Throwable => fail() } } diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index c230a03298..daaa2a0305 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -79,23 +79,25 @@ class ReplSuite extends FunSuite { } test("simple foreach with accumulator") { - val output = runInterpreter("local", """ - |val accum = sc.accumulator(0) - |sc.parallelize(1 to 10).foreach(x => accum += x) - |accum.value - """.stripMargin) + val output = runInterpreter("local", + """ + |val accum = sc.accumulator(0) + |sc.parallelize(1 to 10).foreach(x => accum += x) + |accum.value + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res1: Int = 55", output) } test("external vars") { - val output = runInterpreter("local", """ - |var v = 7 - |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) - |v = 10 - |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) - """.stripMargin) + val output = runInterpreter("local", + """ + |var v = 7 + |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) + |v = 10 + |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Int = 70", output) @@ -103,35 +105,38 @@ class ReplSuite extends FunSuite { } test("external classes") { - val output = runInterpreter("local", """ - |class C { - |def foo = 5 - |} - |sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_) - """.stripMargin) + val output = runInterpreter("local", + """ + |class C { + |def foo = 5 + |} + |sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_) + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Int = 50", output) } test("external functions") { - val output = runInterpreter("local", """ - |def double(x: Int) = x + x - |sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_) - """.stripMargin) + val output = runInterpreter("local", + """ + |def double(x: Int) = x + x + |sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_) + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Int = 110", output) } test("external functions that access vars") { - val output = runInterpreter("local", """ - |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(_+_) - """.stripMargin) + val output = runInterpreter("local", + """ + |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(_+_) + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Int = 70", output) @@ -142,13 +147,14 @@ class ReplSuite extends FunSuite { // 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 // TODO: This doesn't actually work for arrays when we run in local mode! - val output = runInterpreter("local", """ - |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 - """.stripMargin) + val output = runInterpreter("local", + """ + |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 + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Array[Int] = Array(0, 0, 0, 0, 0)", output) @@ -162,12 +168,13 @@ class ReplSuite extends FunSuite { out.write("What's up?\n") out.write("Goodbye\n") out.close() - val output = runInterpreter("local", """ - |var file = sc.textFile("%s/input").cache() - |file.count() - |file.count() - |file.count() - """.stripMargin.format(tempDir.getAbsolutePath)) + val output = runInterpreter("local", + """ + |var file = sc.textFile("%s/input").cache() + |file.count() + |file.count() + |file.count() + """.stripMargin.format(tempDir.getAbsolutePath)) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Long = 3", output) @@ -176,18 +183,19 @@ class ReplSuite extends FunSuite { } 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 - """.stripMargin) + 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 + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Int = 70", output) @@ -198,18 +206,19 @@ class ReplSuite extends FunSuite { if (System.getenv("MESOS_NATIVE_LIBRARY") != null) { test("running on Mesos") { - val output = runInterpreter("localquiet", """ - |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 - """.stripMargin) + val output = runInterpreter("localquiet", + """ + |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 + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Int = 70", output) -- cgit v1.2.3 From db998a6e14389768f93b1fdd6be7847d5f7604fd Mon Sep 17 00:00:00 2001 From: "haitao.yao" Date: Tue, 26 Nov 2013 18:23:48 +0800 Subject: add http timeout for httpbroadcast --- .../main/scala/org/apache/spark/broadcast/HttpBroadcast.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) (limited to 'core/src') 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 609464e38d..47db720416 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -19,6 +19,7 @@ package org.apache.spark.broadcast import java.io.{File, FileOutputStream, ObjectInputStream, OutputStream} import java.net.URL +import java.util.concurrent.TimeUnit import it.unimi.dsi.fastutil.io.FastBufferedInputStream import it.unimi.dsi.fastutil.io.FastBufferedOutputStream @@ -83,6 +84,8 @@ private object HttpBroadcast extends Logging { private val files = new TimeStampedHashSet[String] private val cleaner = new MetadataCleaner(MetadataCleanerType.HTTP_BROADCAST, cleanup) + private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5,TimeUnit.MINUTES).toInt + private lazy val compressionCodec = CompressionCodec.createCodec() def initialize(isDriver: Boolean) { @@ -138,10 +141,13 @@ private object HttpBroadcast extends Logging { def read[T](id: Long): T = { val url = serverUri + "/" + BroadcastBlockId(id).name val in = { + val httpConnection = new URL(url).openConnection() + httpConnection.setReadTimeout(httpReadTimeout) + val inputStream = httpConnection.getInputStream() if (compress) { - compressionCodec.compressedInputStream(new URL(url).openStream()) + compressionCodec.compressedInputStream(inputStream) } else { - new FastBufferedInputStream(new URL(url).openStream(), bufferSize) + new FastBufferedInputStream(inputStream, bufferSize) } } val ser = SparkEnv.get.serializer.newInstance() -- cgit v1.2.3 From 560e44a8e1d5a2cf42bf640090016f6201c6fbd7 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 26 Nov 2013 17:50:29 +0530 Subject: Restored master address for client. --- .../main/scala/org/apache/spark/deploy/client/Client.scala | 11 +++++++---- .../main/scala/org/apache/spark/deploy/master/Master.scala | 4 ++-- 2 files changed, 9 insertions(+), 6 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 408692ec9c..f60e56d959 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -27,7 +27,7 @@ import akka.pattern.AskTimeoutException import akka.pattern.ask import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent, AssociationErrorEvent} -import org.apache.spark.Logging +import org.apache.spark.{SparkException, Logging} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master @@ -49,7 +49,7 @@ private[spark] class Client( val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 - var prevMaster: ActorRef = null // set for unwatching, when it fails. + var masterAddress: Address = null var actor: ActorRef = null var appId: String = null var registered = false @@ -103,11 +103,14 @@ private[spark] class Client( def changeMaster(url: String) { activeMasterUrl = url master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) + masterAddress = activeMasterUrl match { + case Master.sparkUrlRegex(host, port) => Address("akka.tcp", Master.systemName, host, port.toInt) + case x => throw new SparkException("Invalid spark URL:"+x) + } } override def receive = { case RegisteredApplication(appId_, masterUrl) => - prevMaster = sender appId = appId_ registered = true changeMaster(masterUrl) @@ -137,7 +140,7 @@ private[spark] class Client( alreadyDisconnected = false sender ! MasterChangeAcknowledged(appId) - case DisassociatedEvent(_, address, _) => + case DisassociatedEvent(_, address, _) if address == masterAddress => logWarning(s"Connection to $address failed; waiting for master to reconnect...") markDisconnected() 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 81fb5c4e43..0e2b461b13 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 @@ -517,9 +517,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } private[spark] object Master { - private val systemName = "sparkMaster" + val systemName = "sparkMaster" private val actorName = "Master" - private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r + val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) -- cgit v1.2.3 From ed7ecb93ce6ce259eae1f5aeb28e9e336fafa30f Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 26 Nov 2013 13:30:17 -0800 Subject: [SPARK-963] Wait for SparkListenerBus eventQueue to be empty before checking jobLogger state --- .../src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) (limited to 'core/src') 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 984881861c..002368ff55 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.rdd.RDD class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { + val WAIT_TIMEOUT_MILLIS = 10000 test("inner method") { sc = new SparkContext("local", "joblogger") @@ -92,6 +93,8 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } rdd.reduceByKey(_+_).collect() + assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) joblogger.getLogDir should be ("/tmp/spark-%s".format(user)) @@ -120,7 +123,9 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers sc.addSparkListener(joblogger) val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } rdd.reduceByKey(_+_).collect() - + + assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + joblogger.onJobStartCount should be (1) joblogger.onJobEndCount should be (1) joblogger.onTaskEndCount should be (8) -- cgit v1.2.3 From 57579934f0454f258615c10e69ac2adafc5b9835 Mon Sep 17 00:00:00 2001 From: hhd Date: Mon, 25 Nov 2013 17:17:17 -0500 Subject: Emit warning when task size > 100KB --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 15 +++++++++++++++ .../main/scala/org/apache/spark/scheduler/StageInfo.scala | 1 + .../main/scala/org/apache/spark/scheduler/TaskInfo.scala | 2 ++ .../spark/scheduler/cluster/ClusterTaskSetManager.scala | 1 + 4 files changed, 19 insertions(+) (limited to 'core/src') 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 42bb3884c8..4457525ac8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -110,6 +110,9 @@ class DAGScheduler( // resubmit failed stages val POLL_TIMEOUT = 10L + // Warns the user if a stage contains a task with size greater than this value (in KB) + val TASK_SIZE_TO_WARN = 100 + private val eventProcessActor: ActorRef = env.actorSystem.actorOf(Props(new Actor { override def preStart() { context.system.scheduler.schedule(RESUBMIT_TIMEOUT milliseconds, RESUBMIT_TIMEOUT milliseconds) { @@ -430,6 +433,18 @@ class DAGScheduler( handleExecutorLost(execId) case BeginEvent(task, taskInfo) => + for ( + job <- idToActiveJob.get(task.stageId); + stage <- stageIdToStage.get(task.stageId); + stageInfo <- stageToInfos.get(stage) + ) { + if (taskInfo.serializedSize > TASK_SIZE_TO_WARN * 1024 && !stageInfo.emittedTaskSizeWarning) { + stageInfo.emittedTaskSizeWarning = true + logWarning(("Stage %d (%s) contains a task of very large " + + "size (%d KB). The maximum recommended task size is %d KB.").format( + task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, TASK_SIZE_TO_WARN)) + } + } listenerBus.post(SparkListenerTaskStart(task, taskInfo)) case GettingResultEvent(task, taskInfo) => diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 93599dfdc8..e9f2198a00 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -33,4 +33,5 @@ class StageInfo( val name = stage.name val numPartitions = stage.numPartitions val numTasks = stage.numTasks + var emittedTaskSizeWarning = false } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 4bae26f3a6..3c22edd524 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -46,6 +46,8 @@ class TaskInfo( var failed = false + var serializedSize: Int = 0 + def markGettingResult(time: Long = System.currentTimeMillis) { gettingResultTime = time } 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 4c5eca8537..8884ea85a3 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 @@ -377,6 +377,7 @@ 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) + info.serializedSize = serializedTask.limit if (taskAttempts(index).size == 1) taskStarted(task,info) return Some(new TaskDescription(taskId, execId, taskName, index, serializedTask)) -- cgit v1.2.3 From 54862af5ee813030ead80ec097f48620ddb974fc Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 27 Nov 2013 14:26:28 +0530 Subject: Improvements from the review comments and followed Boy Scout Rule. --- .../main/scala/org/apache/spark/MapOutputTracker.scala | 8 ++------ .../scala/org/apache/spark/deploy/master/Master.scala | 8 +++----- .../scala/org/apache/spark/deploy/worker/Worker.scala | 9 +++++++-- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- .../apache/spark/scheduler/cluster/ClusterScheduler.scala | 4 +--- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 5 +---- .../org/apache/spark/storage/BlockManagerMaster.scala | 15 +++------------ core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 5 +++-- docs/configuration.md | 4 ++-- 9 files changed, 23 insertions(+), 37 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index a686b534da..88a7f24884 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -21,15 +21,11 @@ import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashSet +import scala.concurrent.Await +import scala.concurrent.duration._ import akka.actor._ -import scala.concurrent.Await import akka.pattern.ask -import akka.remote._ - -import scala.concurrent.duration.Duration -import akka.util.Timeout -import scala.concurrent.duration._ import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId 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 0e2b461b13..c627dd3806 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 @@ -39,9 +39,6 @@ import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed -import org.apache.spark.deploy.DeployMessages.KillExecutor -import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { import context.dispatcher @@ -159,7 +156,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act System.exit(0) } - case RegisterWorker(id, host, workerPort, cores, memory, webUiPort, publicAddress) => { + case RegisterWorker(id, workerHost, workerPort, cores, memory, workerWebUiPort, publicAddress) => { logInfo("Registering worker %s:%d with %d cores, %s RAM".format( host, workerPort, cores, Utils.megabytesToString(memory))) if (state == RecoveryState.STANDBY) { @@ -167,7 +164,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } else if (idToWorker.contains(id)) { sender ! RegisterWorkerFailed("Duplicate worker ID") } else { - val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress) + val worker = new WorkerInfo(id, workerHost, workerPort, cores, memory, + sender, workerWebUiPort, publicAddress) registerWorker(worker) persistenceEngine.addWorker(worker) sender ! RegisteredWorker(masterUrl, masterWebUiUrl) 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 0a183afd8e..808b54c0af 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,7 +27,7 @@ import scala.concurrent.duration._ import akka.actor._ import akka.remote.{ DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.Logging +import org.apache.spark.{SparkException, Logging} import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master @@ -73,6 +73,7 @@ private[spark] class Worker( val masterLock: Object = new Object() var master: ActorSelection = null + var masterAddress: Address = null var activeMasterUrl: String = "" var activeMasterWebUiUrl : String = "" @volatile var registered = false @@ -136,6 +137,10 @@ private[spark] class Worker( activeMasterUrl = url activeMasterWebUiUrl = uiUrl master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) + masterAddress = activeMasterUrl match { + case Master.sparkUrlRegex(_host, _port) => Address("akka.tcp", Master.systemName, _host, _port.toInt) + case x => throw new SparkException("Invalid spark URL:"+x) + } connected = true } } @@ -240,7 +245,7 @@ private[spark] class Worker( } } - case x: DisassociatedEvent => + case x: DisassociatedEvent if x.remoteAddress == masterAddress => logInfo(s"$x Disassociated !") masterDisconnected() 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 fdea3f6f88..773e9ec182 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -22,7 +22,6 @@ import java.util.Properties import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} -import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ import scala.reflect.ClassTag @@ -115,6 +114,7 @@ class DAGScheduler( private val eventProcessActor: ActorRef = env.actorSystem.actorOf(Props(new Actor { override def preStart() { + import context.dispatcher context.system.scheduler.schedule(RESUBMIT_TIMEOUT, RESUBMIT_TIMEOUT) { if (failed.size > 0) { resubmitFailedStages() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala index 7c9d6a93e4..8056cb2597 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala @@ -24,8 +24,6 @@ import java.util.{TimerTask, Timer} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet - -import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ import org.apache.spark._ @@ -123,7 +121,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) if (System.getProperty("spark.speculation", "false").toBoolean) { logInfo("Starting speculative execution thread") - + import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, SPECULATION_INTERVAL milliseconds) { checkSpeculatableTasks() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index d614dcbdd8..f5e8766f6d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -25,7 +25,7 @@ import scala.concurrent.duration._ import akka.actor._ import akka.pattern.ask -import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent} +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import org.apache.spark.{SparkException, Logging, TaskState} import org.apache.spark.scheduler.TaskDescription @@ -52,7 +52,6 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac private val executorAddress = new HashMap[String, Address] 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] override def preStart() { @@ -77,7 +76,6 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac executorHost(executorId) = Utils.parseHostPort(hostPort)._1 freeCores(executorId) = cores executorAddress(executorId) = sender.path.address - actorToExecutorId(sender) = executorId addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) makeOffers() @@ -147,7 +145,6 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac if (executorActor.contains(executorId)) { logInfo("Executor " + executorId + " disconnected, so removing it") val numCores = freeCores(executorId) - actorToExecutorId -= executorActor(executorId) addressToExecutorId -= executorAddress(executorId) executorActor -= executorId executorHost -= executorId diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index a4aa316e4b..e5de16fc01 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -17,24 +17,15 @@ package org.apache.spark.storage -import akka.actor._ -import scala.concurrent.Await -import scala.concurrent.Future +import scala.concurrent.{Await, Future} +import scala.concurrent.duration._ import scala.concurrent.ExecutionContext.Implicits.global +import akka.actor._ import akka.pattern.ask -import scala.concurrent.duration._ import org.apache.spark.{Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.storage.BlockManagerMessages.GetLocations -import org.apache.spark.storage.BlockManagerMessages.GetLocationsMultipleBlockIds -import org.apache.spark.storage.BlockManagerMessages.RegisterBlockManager -import org.apache.spark.storage.BlockManagerMessages.HeartBeat -import org.apache.spark.storage.BlockManagerMessages.RemoveExecutor -import org.apache.spark.storage.BlockManagerMessages.GetPeers -import org.apache.spark.storage.BlockManagerMessages.RemoveBlock -import org.apache.spark.storage.BlockManagerMessages.RemoveRdd private[spark] class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection]) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 23e9b735f3..3444d8fdfe 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -44,8 +44,9 @@ private[spark] object AkkaUtils { val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - val akkaHeartBeatPauses = System.getProperty("spark.akka.pauses", "60").toInt - val akkaFailureDetector = System.getProperty("spark.akka.failure-detector.threshold", "12.0").toDouble + val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "60").toInt + val akkaFailureDetector = + System.getProperty("spark.akka.failure-detector.threshold", "12.0").toDouble val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "5").toInt val akkaConf = ConfigFactory.parseString( diff --git a/docs/configuration.md b/docs/configuration.md index 25e7cecbfc..4d1a987f64 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -281,7 +281,7 @@ Apart from these, the following properties are also available, and may be useful
- + -- cgit v1.2.3 From 3ec5d7476690a5f8a011ec3eaa2d9003f9559b12 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 28 Nov 2013 13:02:28 +0530 Subject: Fixed the broken build. --- core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 02d75eccc5..688c310ee9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -90,12 +90,13 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = { // Compute the minimum and the maxium val (max: Double, min: Double) = self.mapPartitions { items => - Iterator(items.foldRight(-1/0.0, Double.NaN)((e: Double, x: Pair[Double, Double]) => + Iterator(items.foldRight(Double.NegativeInfinity, + Double.PositiveInfinity)((e: Double, x: Pair[Double, Double]) => (x._1.max(e), x._2.min(e)))) }.reduce { (maxmin1, maxmin2) => (maxmin1._1.max(maxmin2._1), maxmin1._2.min(maxmin2._2)) } - if (max.isNaN() || max.isInfinity || min.isInfinity ) { + if (min.isNaN || max.isNaN || max.isInfinity || min.isInfinity ) { throw new UnsupportedOperationException( "Histogram on either an empty RDD or RDD containing +/-infinity or NaN") } -- cgit v1.2.3 From 18def5d6f20b33c946f9b8b2cea8cfb6848dcc34 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Thu, 28 Nov 2013 17:46:06 +0800 Subject: Bugfix: SPARK-965 & SPARK-966 SPARK-965: https://spark-project.atlassian.net/browse/SPARK-965 SPARK-966: https://spark-project.atlassian.net/browse/SPARK-966 * Add back DAGScheduler.start(), eventProcessActor is created and started here. Notice that function is only called by SparkContext. * Cancel the scheduled stage resubmission task when stopping eventProcessActor * Add a new DAGSchedulerEvent ResubmitFailedStages This event message is sent by the scheduled stage resubmission task to eventProcessActor. In this way, DAGScheduler.resubmitFailedStages is guaranteed to be executed from the same thread that runs DAGScheduler.processEvent. Please refer to discussion in SPARK-966 for details. --- .../main/scala/org/apache/spark/SparkContext.scala | 1 + .../org/apache/spark/scheduler/DAGScheduler.scala | 62 +++++++++++++--------- .../apache/spark/scheduler/DAGSchedulerEvent.scala | 2 + 3 files changed, 40 insertions(+), 25 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3a80241daa..c314f01894 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -270,6 +270,7 @@ class SparkContext( taskScheduler.start() @volatile private[spark] var dagScheduler = new DAGScheduler(taskScheduler) + dagScheduler.start() ui.start() 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 4457525ac8..e2bf08c33f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -113,30 +113,7 @@ class DAGScheduler( // Warns the user if a stage contains a task with size greater than this value (in KB) val TASK_SIZE_TO_WARN = 100 - private val eventProcessActor: ActorRef = env.actorSystem.actorOf(Props(new Actor { - override def preStart() { - context.system.scheduler.schedule(RESUBMIT_TIMEOUT milliseconds, RESUBMIT_TIMEOUT milliseconds) { - if (failed.size > 0) { - resubmitFailedStages() - } - } - } - - /** - * The main event loop of the DAG scheduler, which waits for new-job / task-finished / failure - * events and responds by launching tasks. This runs in a dedicated thread and receives events - * via the eventQueue. - */ - def receive = { - case event: DAGSchedulerEvent => - logDebug("Got event of type " + event.getClass.getName) - - if (!processEvent(event)) - submitWaitingStages() - else - context.stop(self) - } - })) + private var eventProcessActor: ActorRef = _ private[scheduler] val nextJobId = new AtomicInteger(0) @@ -177,6 +154,34 @@ class DAGScheduler( val metadataCleaner = new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup) + def start() { + eventProcessActor = env.actorSystem.actorOf(Props(new Actor { + var resubmissionTask: Cancellable = _ + + override def preStart() { + resubmissionTask = context.system.scheduler.schedule( + RESUBMIT_TIMEOUT.millis, RESUBMIT_TIMEOUT.millis, self, ResubmitFailedStages) + } + + /** + * The main event loop of the DAG scheduler, which waits for new-job / task-finished / failure + * events and responds by launching tasks. This runs in a dedicated thread and receives events + * via the eventQueue. + */ + def receive = { + case event: DAGSchedulerEvent => + logDebug("Got event of type " + event.getClass.getName) + + if (!processEvent(event)) { + submitWaitingStages() + } else { + resubmissionTask.cancel() + context.stop(self) + } + } + })) + } + def addSparkListener(listener: SparkListener) { listenerBus.addListener(listener) } @@ -457,6 +462,11 @@ class DAGScheduler( case TaskSetFailed(taskSet, reason) => abortStage(stageIdToStage(taskSet.stageId), reason) + case ResubmitFailedStages => + if (failed.size > 0) { + resubmitFailedStages() + } + case StopDAGScheduler => // Cancel any active jobs for (job <- activeJobs) { @@ -900,7 +910,9 @@ class DAGScheduler( } def stop() { - eventProcessActor ! StopDAGScheduler + if (eventProcessActor != null) { + eventProcessActor ! StopDAGScheduler + } metadataCleaner.cancel() taskSched.stop() } 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 708d221d60..5353cd24dc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -73,4 +73,6 @@ private[scheduler] case class ExecutorLost(execId: String) extends DAGSchedulerE private[scheduler] case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent +private[scheduler] case object ResubmitFailedStages extends DAGSchedulerEvent + private[scheduler] case object StopDAGScheduler extends DAGSchedulerEvent -- cgit v1.2.3 From 37f161cf6b19eb5b70a251340df0caf21afed84a Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 28 Nov 2013 20:36:18 -0800 Subject: Re-enable zk:// urls for Mesos SparkContexts This was broken in PR #71 when we explicitly disallow anything that didn't fit a mesos:// url. Although it is not really clear that a zk:// url should match Mesos, it is what the docs say and it is necessary for backwards compatibility. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3a80241daa..cf1fd497f0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -162,8 +162,8 @@ class SparkContext( val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r // Regular expression for connecting to Spark deploy clusters val SPARK_REGEX = """spark://(.*)""".r - // Regular expression for connection to Mesos cluster - val MESOS_REGEX = """mesos://(.*)""".r + // Regular expression for connection to Mesos cluster by mesos:// or zk:// url + val MESOS_REGEX = """(mesos|zk)://.*""".r // Regular expression for connection to Simr cluster val SIMR_REGEX = """simr://(.*)""".r @@ -251,14 +251,15 @@ class SparkContext( scheduler.initialize(backend) scheduler - case MESOS_REGEX(mesosUrl) => + case mesosUrl @ MESOS_REGEX(_) => MesosNativeLibrary.load() val scheduler = new ClusterScheduler(this) val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean + val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { - new CoarseMesosSchedulerBackend(scheduler, this, mesosUrl, appName) + new CoarseMesosSchedulerBackend(scheduler, this, url, appName) } else { - new MesosSchedulerBackend(scheduler, this, mesosUrl, appName) + new MesosSchedulerBackend(scheduler, this, url, appName) } scheduler.initialize(backend) scheduler -- cgit v1.2.3 From 081a0b6861321d262a82166bc1df61959e9c6387 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 28 Nov 2013 20:39:10 -0800 Subject: Add unit test for SparkContext scheduler creation Since YARN and Mesos are not necessarily available in the system, they are allowed to pass as long as the YARN/Mesos code paths are exercised. --- .../main/scala/org/apache/spark/SparkContext.scala | 234 +++++++++++---------- .../spark/scheduler/local/LocalScheduler.scala | 2 +- .../spark/SparkContextSchedulerCreationSuite.scala | 135 ++++++++++++ 3 files changed, 255 insertions(+), 116 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index cf1fd497f0..1eb00e79e1 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -153,121 +153,7 @@ class SparkContext( executorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler - private[spark] var taskScheduler: TaskScheduler = { - // Regular expression used for local[N] master format - val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r - // Regular expression for local[N, maxRetries], used in tests with failing tasks - val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r - // Regular expression for simulating a Spark cluster of [N, cores, memory] locally - val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r - // Regular expression for connecting to Spark deploy clusters - val SPARK_REGEX = """spark://(.*)""".r - // Regular expression for connection to Mesos cluster by mesos:// or zk:// url - val MESOS_REGEX = """(mesos|zk)://.*""".r - // Regular expression for connection to Simr cluster - val SIMR_REGEX = """simr://(.*)""".r - - master match { - case "local" => - new LocalScheduler(1, 0, this) - - case LOCAL_N_REGEX(threads) => - new LocalScheduler(threads.toInt, 0, this) - - case LOCAL_N_FAILURES_REGEX(threads, maxFailures) => - new LocalScheduler(threads.toInt, maxFailures.toInt, this) - - case SPARK_REGEX(sparkUrl) => - val scheduler = new ClusterScheduler(this) - val masterUrls = sparkUrl.split(",").map("spark://" + _) - val backend = new SparkDeploySchedulerBackend(scheduler, this, masterUrls, appName) - scheduler.initialize(backend) - scheduler - - case SIMR_REGEX(simrUrl) => - val scheduler = new ClusterScheduler(this) - val backend = new SimrSchedulerBackend(scheduler, this, simrUrl) - scheduler.initialize(backend) - scheduler - - case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => - // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang. - val memoryPerSlaveInt = memoryPerSlave.toInt - if (SparkContext.executorMemoryRequested > memoryPerSlaveInt) { - throw new SparkException( - "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format( - memoryPerSlaveInt, SparkContext.executorMemoryRequested)) - } - - val scheduler = new ClusterScheduler(this) - val localCluster = new LocalSparkCluster( - numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) - val masterUrls = localCluster.start() - val backend = new SparkDeploySchedulerBackend(scheduler, this, masterUrls, appName) - scheduler.initialize(backend) - backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { - localCluster.stop() - } - scheduler - - case "yarn-standalone" => - val scheduler = try { - val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler") - val cons = clazz.getConstructor(classOf[SparkContext]) - cons.newInstance(this).asInstanceOf[ClusterScheduler] - } catch { - // TODO: Enumerate the exact reasons why it can fail - // But irrespective of it, it means we cannot proceed ! - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) - } - } - val backend = new CoarseGrainedSchedulerBackend(scheduler, this.env.actorSystem) - scheduler.initialize(backend) - scheduler - - case "yarn-client" => - val scheduler = try { - val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") - val cons = clazz.getConstructor(classOf[SparkContext]) - cons.newInstance(this).asInstanceOf[ClusterScheduler] - - } catch { - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) - } - } - - val backend = try { - val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend") - val cons = clazz.getConstructor(classOf[ClusterScheduler], classOf[SparkContext]) - cons.newInstance(scheduler, this).asInstanceOf[CoarseGrainedSchedulerBackend] - } catch { - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) - } - } - - scheduler.initialize(backend) - scheduler - - case mesosUrl @ MESOS_REGEX(_) => - MesosNativeLibrary.load() - val scheduler = new ClusterScheduler(this) - val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean - val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs - val backend = if (coarseGrained) { - new CoarseMesosSchedulerBackend(scheduler, this, url, appName) - } else { - new MesosSchedulerBackend(scheduler, this, url, appName) - } - scheduler.initialize(backend) - scheduler - - case _ => - throw new SparkException("Could not parse Master URL: '" + master + "'") - } - } + private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master, appName) taskScheduler.start() @volatile private[spark] var dagScheduler = new DAGScheduler(taskScheduler) @@ -1137,6 +1023,124 @@ object SparkContext { .map(Utils.memoryStringToMb) .getOrElse(512) } + + // Creates a task scheduler based on a given master URL. Extracted for testing. + private + def createTaskScheduler(sc: SparkContext, master: String, appName: String): TaskScheduler = { + // Regular expression used for local[N] master format + val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r + // Regular expression for local[N, maxRetries], used in tests with failing tasks + val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r + // Regular expression for simulating a Spark cluster of [N, cores, memory] locally + val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r + // Regular expression for connecting to Spark deploy clusters + val SPARK_REGEX = """spark://(.*)""".r + // Regular expression for connection to Mesos cluster by mesos:// or zk:// url + val MESOS_REGEX = """(mesos|zk)://.*""".r + // Regular expression for connection to Simr cluster + val SIMR_REGEX = """simr://(.*)""".r + + master match { + case "local" => + new LocalScheduler(1, 0, sc) + + case LOCAL_N_REGEX(threads) => + new LocalScheduler(threads.toInt, 0, sc) + + case LOCAL_N_FAILURES_REGEX(threads, maxFailures) => + new LocalScheduler(threads.toInt, maxFailures.toInt, sc) + + case SPARK_REGEX(sparkUrl) => + val scheduler = new ClusterScheduler(sc) + val masterUrls = sparkUrl.split(",").map("spark://" + _) + val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName) + scheduler.initialize(backend) + scheduler + + case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => + // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang. + val memoryPerSlaveInt = memoryPerSlave.toInt + if (SparkContext.executorMemoryRequested > memoryPerSlaveInt) { + throw new SparkException( + "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format( + memoryPerSlaveInt, SparkContext.executorMemoryRequested)) + } + + val scheduler = new ClusterScheduler(sc) + val localCluster = new LocalSparkCluster( + numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) + val masterUrls = localCluster.start() + val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName) + scheduler.initialize(backend) + backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { + localCluster.stop() + } + scheduler + + case "yarn-standalone" => + val scheduler = try { + val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler") + val cons = clazz.getConstructor(classOf[SparkContext]) + cons.newInstance(sc).asInstanceOf[ClusterScheduler] + } catch { + // TODO: Enumerate the exact reasons why it can fail + // But irrespective of it, it means we cannot proceed ! + case th: Throwable => { + throw new SparkException("YARN mode not available ?", th) + } + } + val backend = new CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + scheduler.initialize(backend) + scheduler + + case "yarn-client" => + val scheduler = try { + val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") + val cons = clazz.getConstructor(classOf[SparkContext]) + cons.newInstance(sc).asInstanceOf[ClusterScheduler] + + } catch { + case th: Throwable => { + throw new SparkException("YARN mode not available ?", th) + } + } + + val backend = try { + val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend") + val cons = clazz.getConstructor(classOf[ClusterScheduler], classOf[SparkContext]) + cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] + } catch { + case th: Throwable => { + throw new SparkException("YARN mode not available ?", th) + } + } + + scheduler.initialize(backend) + scheduler + + case mesosUrl @ MESOS_REGEX(_) => + MesosNativeLibrary.load() + val scheduler = new ClusterScheduler(sc) + val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean + val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs + val backend = if (coarseGrained) { + new CoarseMesosSchedulerBackend(scheduler, sc, url, appName) + } else { + new MesosSchedulerBackend(scheduler, sc, url, appName) + } + scheduler.initialize(backend) + scheduler + + case SIMR_REGEX(simrUrl) => + val scheduler = new ClusterScheduler(sc) + val backend = new SimrSchedulerBackend(scheduler, sc, simrUrl) + scheduler.initialize(backend) + scheduler + + case _ => + throw new SparkException("Could not parse Master URL: '" + master + "'") + } + } } /** 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 2699f0b33e..5af51164f7 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 @@ -74,7 +74,7 @@ class LocalActor(localScheduler: LocalScheduler, private var freeCores: Int) } } -private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: SparkContext) +private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val sc: SparkContext) extends TaskScheduler with ExecutorBackend with Logging { diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala new file mode 100644 index 0000000000..61d6163659 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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, PrivateMethodTester} + +import org.apache.spark.scheduler.TaskScheduler +import org.apache.spark.scheduler.cluster.{ClusterScheduler, SimrSchedulerBackend, SparkDeploySchedulerBackend} +import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} +import org.apache.spark.scheduler.local.LocalScheduler + +class SparkContextSchedulerCreationSuite + extends FunSuite with PrivateMethodTester with LocalSparkContext with Logging { + + def createTaskScheduler(master: String): TaskScheduler = { + // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the + // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. + sc = new SparkContext("local", "test") + val createTaskSchedulerMethod = PrivateMethod[TaskScheduler]('createTaskScheduler) + SparkContext invokePrivate createTaskSchedulerMethod(sc, master, "test") + } + + test("bad-master") { + val e = intercept[SparkException] { + createTaskScheduler("localhost:1234") + } + assert(e.getMessage.contains("Could not parse Master URL")) + } + + test("local") { + createTaskScheduler("local") match { + case s: LocalScheduler => + assert(s.threads === 1) + assert(s.maxFailures === 0) + case _ => fail() + } + } + + test("local-n") { + createTaskScheduler("local[5]") match { + case s: LocalScheduler => + assert(s.threads === 5) + assert(s.maxFailures === 0) + case _ => fail() + } + } + + test("local-n-failures") { + createTaskScheduler("local[4, 2]") match { + case s: LocalScheduler => + assert(s.threads === 4) + assert(s.maxFailures === 2) + case _ => fail() + } + } + + test("simr") { + createTaskScheduler("simr://uri") match { + case s: ClusterScheduler => + assert(s.backend.isInstanceOf[SimrSchedulerBackend]) + case _ => fail() + } + } + + test("local-cluster") { + createTaskScheduler("local-cluster[3, 14, 512]") match { + case s: ClusterScheduler => + assert(s.backend.isInstanceOf[SparkDeploySchedulerBackend]) + case _ => fail() + } + } + + def testYarn(master: String, expectedClassName: String) { + try { + createTaskScheduler(master) match { + case s: ClusterScheduler => + assert(s.getClass === Class.forName(expectedClassName)) + case _ => fail() + } + } catch { + case e: SparkException => + assert(e.getMessage.contains("YARN mode not available")) + logWarning("YARN not available, could not test actual YARN scheduler creation") + case e: Throwable => fail(e) + } + } + test("yarn-standalone") { + testYarn("yarn-standalone", "org.apache.spark.scheduler.cluster.YarnClusterScheduler") + } + test("yarn-client") { + testYarn("yarn-client", "org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") + } + + def testMesos(master: String, expectedClass: Class[_]) { + try { + createTaskScheduler(master) match { + case s: ClusterScheduler => + assert(s.backend.getClass === expectedClass) + case _ => fail() + } + } catch { + case e: UnsatisfiedLinkError => + assert(e.getMessage.contains("no mesos in")) + logWarning("Mesos not available, could not test actual Mesos scheduler creation") + case e: Throwable => fail(e) + } + } + test("mesos fine-grained") { + System.setProperty("spark.mesos.coarse", "false") + testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend]) + } + test("mesos coarse-grained") { + System.setProperty("spark.mesos.coarse", "true") + testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend]) + } + test("mesos with zookeeper") { + System.setProperty("spark.mesos.coarse", "false") + testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend]) + } +} -- cgit v1.2.3 From 1e25086009ff6421790609e406d00e1b978d6dbe Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Fri, 29 Nov 2013 15:56:47 +0800 Subject: Updated some inline comments in DAGScheduler --- .../org/apache/spark/scheduler/DAGScheduler.scala | 31 ++++++++++++++++++---- 1 file changed, 26 insertions(+), 5 deletions(-) (limited to 'core/src') 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 e2bf08c33f..08cf76325b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -154,24 +154,43 @@ class DAGScheduler( val metadataCleaner = new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup) + /** + * Starts the event processing actor. The actor has two responsibilities: + * + * 1. Waits for events like job submission, task finished, task failure etc., and calls + * [[org.apache.spark.scheduler.DAGScheduler.processEvent()]] to process them. + * 2. Schedules a periodical task to resubmit failed stages. + * + * NOTE: the actor cannot be started in the constructor, because the periodical task references + * some internal states of the enclosing [[org.apache.spark.scheduler.DAGScheduler]] object, thus + * cannot be scheduled until the [[org.apache.spark.scheduler.DAGScheduler]] is fully constructed. + */ def start() { eventProcessActor = env.actorSystem.actorOf(Props(new Actor { var resubmissionTask: Cancellable = _ override def preStart() { + /** + * A message is sent to the actor itself periodically to remind the actor to resubmit failed + * stages. In this way, stage resubmission can be done within the same thread context of + * other event processing logic to avoid unnecessary synchronization overhead. + */ resubmissionTask = context.system.scheduler.schedule( RESUBMIT_TIMEOUT.millis, RESUBMIT_TIMEOUT.millis, self, ResubmitFailedStages) } /** - * The main event loop of the DAG scheduler, which waits for new-job / task-finished / failure - * events and responds by launching tasks. This runs in a dedicated thread and receives events - * via the eventQueue. + * The main event loop of the DAG scheduler. */ def receive = { case event: DAGSchedulerEvent => logDebug("Got event of type " + event.getClass.getName) + /** + * All events are forwarded to `processEvent()`, so that the event processing logic can + * easily tested without starting a dedicated actor. Please refer to `DAGSchedulerSuite` + * for details. + */ if (!processEvent(event)) { submitWaitingStages() } else { @@ -383,8 +402,10 @@ class DAGScheduler( } /** - * Process one event retrieved from the event queue. - * Returns true if we should stop the event loop. + * Process one event retrieved from the event processing actor. + * + * @param event The event to be processed. + * @return `true` if we should stop the event loop. */ private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = { event match { -- cgit v1.2.3 From 4a1d966e26e56fc5d42a828f414b4eca433c3a22 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Fri, 29 Nov 2013 16:02:58 +0800 Subject: More comments --- core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 +++ 1 file changed, 3 insertions(+) (limited to 'core/src') 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 08cf76325b..bc37a70e98 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -167,6 +167,9 @@ class DAGScheduler( */ def start() { eventProcessActor = env.actorSystem.actorOf(Props(new Actor { + /** + * A handle to the periodical task, used to cancel the task when the actor is stopped. + */ var resubmissionTask: Cancellable = _ override def preStart() { -- cgit v1.2.3 From 1bc83ca79187979f58385d3f28236111217174e0 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 28 Nov 2013 16:07:15 +0530 Subject: Changed defaults for akka to almost disable failure detector. --- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 3444d8fdfe..5df8213d74 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -39,21 +39,22 @@ private[spark] object AkkaUtils { 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", "60").toInt + val akkaTimeout = System.getProperty("spark.akka.timeout", "100").toInt val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "60").toInt + val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "600").toInt val akkaFailureDetector = - System.getProperty("spark.akka.failure-detector.threshold", "12.0").toDouble - val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "5").toInt + System.getProperty("spark.akka.failure-detector.threshold", "300.0").toDouble + val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "1000").toInt val akkaConf = ConfigFactory.parseString( s""" |akka.daemonic = on |akka.loggers = [""akka.event.slf4j.Slf4jLogger""] |akka.stdout-loglevel = "ERROR" + |akka.jvm-exit-on-fatal-error = off |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector @@ -61,6 +62,7 @@ private[spark] object AkkaUtils { |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" |akka.remote.netty.tcp.hostname = "$host" |akka.remote.netty.tcp.port = $port + |akka.remote.netty.tcp.tcp-nodelay = on |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}MiB |akka.remote.netty.tcp.execution-pool-size = $akkaThreads -- cgit v1.2.3 From 96df26be474abe89a8dd83da08fc27903ad19ba7 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 29 Nov 2013 13:20:35 -0800 Subject: Add spaces between tests --- .../scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala | 5 +++++ 1 file changed, 5 insertions(+) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 61d6163659..151af0d213 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -99,9 +99,11 @@ class SparkContextSchedulerCreationSuite case e: Throwable => fail(e) } } + test("yarn-standalone") { testYarn("yarn-standalone", "org.apache.spark.scheduler.cluster.YarnClusterScheduler") } + test("yarn-client") { testYarn("yarn-client", "org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") } @@ -120,14 +122,17 @@ class SparkContextSchedulerCreationSuite case e: Throwable => fail(e) } } + test("mesos fine-grained") { System.setProperty("spark.mesos.coarse", "false") testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend]) } + test("mesos coarse-grained") { System.setProperty("spark.mesos.coarse", "true") testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend]) } + test("mesos with zookeeper") { System.setProperty("spark.mesos.coarse", "false") testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend]) -- cgit v1.2.3 From 4d53830eb79174cfd9641f6342727bc980d5c3e0 Mon Sep 17 00:00:00 2001 From: Sundeep Narravula Date: Sat, 30 Nov 2013 16:18:12 -0800 Subject: Scheduler quits when createStage fails. The current scheduler thread does not handle exceptions from createStage stage while launching new jobs. The thread fails on any exception that gets triggered at that level, leaving the cluster hanging with no schduler. --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) (limited to 'core/src') 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 4457525ac8..f6a4482679 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -384,7 +384,15 @@ class DAGScheduler( private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = { event match { case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => - val finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite)) + var finalStage:Stage = null + try { + finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite)) + } catch { + case e: Exception => + logWarning("Creating new stage failed due to exception - job: " + jobId ) + listener.jobFailed(e) + return false + } val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() logInfo("Got job " + job.jobId + " (" + callSite + ") with " + partitions.length + -- cgit v1.2.3 From 9cf7f31e4d4e542b88b6a474bdf08d07fdd3652c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 30 Nov 2013 18:07:36 -0800 Subject: Memoize preferred locations in ZippedPartitionsBaseRDD so preferred location computation doesn't lead to exponential explosion. (cherry picked from commit e36fe55a031d2c01c9d7c5d85965951c681a0c74) Signed-off-by: Reynold Xin --- .../org/apache/spark/rdd/ZippedPartitionsRDD.scala | 27 +++++++++------------- 1 file changed, 11 insertions(+), 16 deletions(-) (limited to 'core/src') 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 faeb316664..a97d2a01c8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -22,7 +22,8 @@ import java.io.{ObjectOutputStream, IOException} private[spark] class ZippedPartitionsPartition( idx: Int, - @transient rdds: Seq[RDD[_]]) + @transient rdds: Seq[RDD[_]], + @transient val preferredLocations: Seq[String]) extends Partition { override val index: Int = idx @@ -47,27 +48,21 @@ abstract class ZippedPartitionsBaseRDD[V: ClassManifest]( if (preservesPartitioning) firstParent[Any].partitioner else None override def getPartitions: Array[Partition] = { - val sizes = rdds.map(x => x.partitions.size) - if (!sizes.forall(x => x == sizes(0))) { + val numParts = rdds.head.partitions.size + if (!rdds.forall(rdd => rdd.partitions.size == numParts)) { throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions") } - val array = new Array[Partition](sizes(0)) - for (i <- 0 until sizes(0)) { - array(i) = new ZippedPartitionsPartition(i, rdds) + Array.tabulate[Partition](numParts) { i => + val prefs = rdds.map(rdd => rdd.preferredLocations(rdd.partitions(i))) + // Check whether there are any hosts that match all RDDs; otherwise return the union + val exactMatchLocations = prefs.reduce((x, y) => x.intersect(y)) + val locs = if (!exactMatchLocations.isEmpty) exactMatchLocations else prefs.flatten.distinct + new ZippedPartitionsPartition(i, rdds, locs) } - array } override def getPreferredLocations(s: Partition): Seq[String] = { - 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 - } + s.asInstanceOf[ZippedPartitionsPartition].preferredLocations } override def clearDependencies() { -- cgit v1.2.3 From be3ea2394fa2e626fb6b5f2cd46e7156016c9b3f Mon Sep 17 00:00:00 2001 From: Sundeep Narravula Date: Sun, 1 Dec 2013 00:50:34 -0800 Subject: Log exception in scheduler in addition to passing it to the caller. Code Styling changes. --- core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) (limited to 'core/src') 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 f6a4482679..915918630b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -384,12 +384,14 @@ class DAGScheduler( private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = { event match { case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => - var finalStage:Stage = null + var finalStage: Stage = null try { + // New stage creation at times and if its not protected, the scheduler thread is killed. + // e.g. it can fail when jobs are run on HadoopRDD whose underlying hdfs files have been deleted finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite)) } catch { case e: Exception => - logWarning("Creating new stage failed due to exception - job: " + jobId ) + logWarning("Creating new stage failed due to exception - job: " + jobId, e) listener.jobFailed(e) return false } -- cgit v1.2.3 From 5b11028a0479623f41e95a41825a9bdfc944b323 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 2 Dec 2013 10:41:26 +0530 Subject: Made akka capable of tolerating fatal exceptions and moving on. --- .../scala/org/apache/spark/util/AkkaUtils.scala | 4 +- .../org/apache/spark/util/SparkActorSystem.scala | 112 +++++++++++++++++++++ 2 files changed, 114 insertions(+), 2 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 5df8213d74..407e9ffe90 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import akka.actor.{ActorSystem, ExtendedActorSystem} +import akka.actor.{SparkActorSystem, ActorSystem, ExtendedActorSystem} import com.typesafe.config.ConfigFactory import scala.concurrent.duration._ import scala.concurrent.Await @@ -70,7 +70,7 @@ private[spark] object AkkaUtils { |akka.remote.log-remote-lifecycle-events = $lifecycleEvents """.stripMargin) - val actorSystem = ActorSystem(name, akkaConf) + val actorSystem = SparkActorSystem(name, akkaConf) val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.getDefaultAddress.port.get diff --git a/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala b/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala new file mode 100644 index 0000000000..461e7ab08f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala @@ -0,0 +1,112 @@ +/** + * Copyright (C) 2009-2013 Typesafe Inc. + */ + +package akka.actor + +import com.typesafe.config.Config +import akka.util._ +import scala.util.control.{NonFatal, ControlThrowable} + +/** + * An actorSystem specific to spark. It has an additional feature of letting spark tolerate + * fatal exceptions. + */ +object SparkActorSystem { + + def apply(name: String, config: Config): ActorSystem = apply(name, config, findClassLoader()) + + def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = + new SparkActorSystemImpl(name, config, classLoader).start() + + /** + * INTERNAL API + */ + private[akka] def findClassLoader(): ClassLoader = { + def findCaller(get: Int ⇒ Class[_]): ClassLoader = + Iterator.from(2 /*is the magic number, promise*/).map(get) dropWhile { + c ⇒ + c != null && + (c.getName.startsWith("akka.actor.ActorSystem") || + c.getName.startsWith("scala.Option") || + c.getName.startsWith("scala.collection.Iterator") || + c.getName.startsWith("akka.util.Reflect")) + } next() match { + case null ⇒ getClass.getClassLoader + case c ⇒ c.getClassLoader + } + + Option(Thread.currentThread.getContextClassLoader) orElse + (Reflect.getCallerClass map findCaller) getOrElse + getClass.getClassLoader + } +} + +private[akka] class SparkActorSystemImpl(override val name: String, + applicationConfig: Config, + classLoader: ClassLoader) + extends ActorSystemImpl(name, applicationConfig, classLoader) { + + protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = + new Thread.UncaughtExceptionHandler() { + def uncaughtException(thread: Thread, cause: Throwable): Unit = { + cause match { + case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable + ⇒ log.error(cause, "Uncaught error from thread [{}]", thread.getName) + case _ ⇒ + if (settings.JvmExitOnFatalError) { + try { + log.error(cause, "Uncaught error from thread [{}] shutting down JVM since " + + "'akka.jvm-exit-on-fatal-error' is enabled", thread.getName) + import System.err + err.print("Uncaught error from thread [") + err.print(thread.getName) + err.print("] shutting down JVM since 'akka.jvm-exit-on-fatal-error' is enabled for " + + "ActorSystem[") + err.print(name) + err.println("]") + cause.printStackTrace(System.err) + System.err.flush() + } finally { + System.exit(-1) + } + } else { + log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + + "ActorSystem tolerating and continuing.... [{}]", thread.getName, name) + //shutdown() //TODO make it configurable + if (thread.isAlive) log.error("Thread is still alive") + else { + log.error("Thread is dead") + } + } + } + } + } + + override def stop(actor: ActorRef): Unit = { + val path = actor.path + val guard = guardian.path + val sys = systemGuardian.path + path.parent match { + case `guard` ⇒ guardian ! StopChild(actor) + case `sys` ⇒ systemGuardian ! StopChild(actor) + case _ ⇒ actor.asInstanceOf[InternalActorRef].stop() + } + } + + + override def /(actorName: String): ActorPath = guardian.path / actorName + + override def /(path: Iterable[String]): ActorPath = guardian.path / path + + private lazy val _start: this.type = { + // the provider is expected to start default loggers, LocalActorRefProvider does this + provider.init(this) + this + } + + override def start(): this.type = _start + + override def toString: String = lookupRoot.path.root.address.toString + +} -- cgit v1.2.3 From f6c8c1c7b686a010ffcec238db14eda34f1645f1 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 2 Dec 2013 11:42:53 -0800 Subject: Cleanup and documentation of SparkActorSystem --- .../org/apache/spark/util/SparkActorSystem.scala | 114 ++++++--------------- 1 file changed, 29 insertions(+), 85 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala b/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala index 461e7ab08f..a679fd6142 100644 --- a/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala +++ b/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala @@ -2,111 +2,55 @@ * Copyright (C) 2009-2013 Typesafe Inc. */ +// Must be in akka.actor package as ActorSystemImpl is protected[akka]. package akka.actor +import scala.util.control.{ControlThrowable, NonFatal} + import com.typesafe.config.Config -import akka.util._ -import scala.util.control.{NonFatal, ControlThrowable} /** - * An actorSystem specific to spark. It has an additional feature of letting spark tolerate - * fatal exceptions. + * An ActorSystem specific to Spark. Based off of [[akka.actor.ActorSystem]]. + * The only change from the default system is that we do not shut down the ActorSystem + * in the event of a fatal exception. This is necessary as Spark is allowed to recover + * from fatal exceptions (see [[org.apache.spark.executor.Executor]]). */ object SparkActorSystem { - - def apply(name: String, config: Config): ActorSystem = apply(name, config, findClassLoader()) + def apply(name: String, config: Config): ActorSystem = + apply(name, config, ActorSystem.findClassLoader()) def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = new SparkActorSystemImpl(name, config, classLoader).start() - - /** - * INTERNAL API - */ - private[akka] def findClassLoader(): ClassLoader = { - def findCaller(get: Int ⇒ Class[_]): ClassLoader = - Iterator.from(2 /*is the magic number, promise*/).map(get) dropWhile { - c ⇒ - c != null && - (c.getName.startsWith("akka.actor.ActorSystem") || - c.getName.startsWith("scala.Option") || - c.getName.startsWith("scala.collection.Iterator") || - c.getName.startsWith("akka.util.Reflect")) - } next() match { - case null ⇒ getClass.getClassLoader - case c ⇒ c.getClassLoader - } - - Option(Thread.currentThread.getContextClassLoader) orElse - (Reflect.getCallerClass map findCaller) getOrElse - getClass.getClassLoader - } } -private[akka] class SparkActorSystemImpl(override val name: String, - applicationConfig: Config, - classLoader: ClassLoader) +private[akka] class SparkActorSystemImpl( + override val name: String, + applicationConfig: Config, + classLoader: ClassLoader) extends ActorSystemImpl(name, applicationConfig, classLoader) { - protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = + protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = { + val fallbackHandler = super.uncaughtExceptionHandler + new Thread.UncaughtExceptionHandler() { def uncaughtException(thread: Thread, cause: Throwable): Unit = { - cause match { - case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable - ⇒ log.error(cause, "Uncaught error from thread [{}]", thread.getName) - case _ ⇒ - if (settings.JvmExitOnFatalError) { - try { - log.error(cause, "Uncaught error from thread [{}] shutting down JVM since " + - "'akka.jvm-exit-on-fatal-error' is enabled", thread.getName) - import System.err - err.print("Uncaught error from thread [") - err.print(thread.getName) - err.print("] shutting down JVM since 'akka.jvm-exit-on-fatal-error' is enabled for " + - "ActorSystem[") - err.print(name) - err.println("]") - cause.printStackTrace(System.err) - System.err.flush() - } finally { - System.exit(-1) - } - } else { - log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + - "ActorSystem tolerating and continuing.... [{}]", thread.getName, name) - //shutdown() //TODO make it configurable - if (thread.isAlive) log.error("Thread is still alive") - else { - log.error("Thread is dead") - } - } + if (isFatalError(cause) && !settings.JvmExitOnFatalError) { + log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + + "ActorSystem tolerating and continuing.... [{}]", thread.getName, name) + //shutdown() //TODO make it configurable + } else { + fallbackHandler.uncaughtException(thread, cause) } } } - - override def stop(actor: ActorRef): Unit = { - val path = actor.path - val guard = guardian.path - val sys = systemGuardian.path - path.parent match { - case `guard` ⇒ guardian ! StopChild(actor) - case `sys` ⇒ systemGuardian ! StopChild(actor) - case _ ⇒ actor.asInstanceOf[InternalActorRef].stop() - } } - - override def /(actorName: String): ActorPath = guardian.path / actorName - - override def /(path: Iterable[String]): ActorPath = guardian.path / path - - private lazy val _start: this.type = { - // the provider is expected to start default loggers, LocalActorRefProvider does this - provider.init(this) - this + def isFatalError(e: Throwable): Boolean = { + e match { + case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => + false + case _ => + true + } } - - override def start(): this.type = _start - - override def toString: String = lookupRoot.path.root.address.toString - } -- cgit v1.2.3 From 58b3aff9a871a38446aacc2d60b65199d44e56bb Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Mon, 2 Dec 2013 20:30:03 -0800 Subject: Fixed problem with scheduler delay --- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) (limited to 'core/src') 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 fc8c334cb5..8deb495068 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 @@ -120,11 +120,14 @@ private[spark] class StagePage(parent: JobProgressUI) { // machine and to send back the result (but not the time to fetch the task result, // if it needed to be fetched from the block manager on the worker). val schedulerDelays = validTasks.map{case (info, metrics, exception) => - if (info.gettingResultTime > 0) { - (info.gettingResultTime - info.launchTime).toDouble - } else { - (info.finishTime - info.launchTime).toDouble + val totalExecutionTime = { + if (info.gettingResultTime > 0) { + (info.gettingResultTime - info.launchTime).toDouble + } else { + (info.finishTime - info.launchTime).toDouble + } } + totalExecutionTime - metrics.get.executorRunTime } val schedulerDelayQuantiles = ("Scheduler delay" +: Distribution(schedulerDelays).get.getQuantiles().map( -- cgit v1.2.3 From e34b4693d380c39d4a142515e416588e63d06297 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 2 Dec 2013 21:24:44 -0800 Subject: Mark partitioner, name, and generator field in RDD as @transient. --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 5b1285307d..96e4841c78 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -101,7 +101,7 @@ abstract class RDD[T: ClassManifest]( protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ - val partitioner: Option[Partitioner] = None + @transient val partitioner: Option[Partitioner] = None // ======================================================================= // Methods and fields available on all RDDs @@ -114,7 +114,7 @@ abstract class RDD[T: ClassManifest]( val id: Int = sc.newRddId() /** A friendly name for this RDD */ - var name: String = null + @transient var name: String = null /** Assign a name to this RDD */ def setName(_name: String) = { @@ -123,7 +123,7 @@ abstract class RDD[T: ClassManifest]( } /** User-defined generator of this RDD*/ - var generator = Utils.getCallSiteInfo.firstUserClass + @transient var generator = Utils.getCallSiteInfo.firstUserClass /** Reset generator*/ def setGenerator(_generator: String) = { -- cgit v1.2.3 From 0f24576c08a361f323b7ad9babfd5d8431d57df0 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 2 Dec 2013 11:42:53 -0800 Subject: Cleanup and documentation of SparkActorSystem --- .../org/apache/spark/util/SparkActorSystem.scala | 114 ++++++--------------- 1 file changed, 29 insertions(+), 85 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala b/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala index 461e7ab08f..d329063e43 100644 --- a/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala +++ b/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala @@ -2,111 +2,55 @@ * Copyright (C) 2009-2013 Typesafe Inc. */ +// Must be in akka.actor package as ActorSystemImpl is protected[akka]. package akka.actor +import scala.util.control.{ControlThrowable, NonFatal} + import com.typesafe.config.Config -import akka.util._ -import scala.util.control.{NonFatal, ControlThrowable} /** - * An actorSystem specific to spark. It has an additional feature of letting spark tolerate - * fatal exceptions. + * An ActorSystem specific to Spark. Based off of [[akka.actor.ActorSystem]]. + * The only change from the default system is that we do not shut down the ActorSystem + * in the event of a fatal exception. This is necessary as Spark is allowed to recover + * from fatal exceptions (see [[org.apache.spark.executor.Executor]]). */ object SparkActorSystem { - - def apply(name: String, config: Config): ActorSystem = apply(name, config, findClassLoader()) + def apply(name: String, config: Config): ActorSystem = + apply(name, config, ActorSystem.findClassLoader()) def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = new SparkActorSystemImpl(name, config, classLoader).start() - - /** - * INTERNAL API - */ - private[akka] def findClassLoader(): ClassLoader = { - def findCaller(get: Int ⇒ Class[_]): ClassLoader = - Iterator.from(2 /*is the magic number, promise*/).map(get) dropWhile { - c ⇒ - c != null && - (c.getName.startsWith("akka.actor.ActorSystem") || - c.getName.startsWith("scala.Option") || - c.getName.startsWith("scala.collection.Iterator") || - c.getName.startsWith("akka.util.Reflect")) - } next() match { - case null ⇒ getClass.getClassLoader - case c ⇒ c.getClassLoader - } - - Option(Thread.currentThread.getContextClassLoader) orElse - (Reflect.getCallerClass map findCaller) getOrElse - getClass.getClassLoader - } } -private[akka] class SparkActorSystemImpl(override val name: String, - applicationConfig: Config, - classLoader: ClassLoader) +private[akka] class SparkActorSystemImpl( + override val name: String, + applicationConfig: Config, + classLoader: ClassLoader) extends ActorSystemImpl(name, applicationConfig, classLoader) { - protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = + protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = { + val fallbackHandler = super.uncaughtExceptionHandler + new Thread.UncaughtExceptionHandler() { def uncaughtException(thread: Thread, cause: Throwable): Unit = { - cause match { - case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable - ⇒ log.error(cause, "Uncaught error from thread [{}]", thread.getName) - case _ ⇒ - if (settings.JvmExitOnFatalError) { - try { - log.error(cause, "Uncaught error from thread [{}] shutting down JVM since " + - "'akka.jvm-exit-on-fatal-error' is enabled", thread.getName) - import System.err - err.print("Uncaught error from thread [") - err.print(thread.getName) - err.print("] shutting down JVM since 'akka.jvm-exit-on-fatal-error' is enabled for " + - "ActorSystem[") - err.print(name) - err.println("]") - cause.printStackTrace(System.err) - System.err.flush() - } finally { - System.exit(-1) - } - } else { - log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + - "ActorSystem tolerating and continuing.... [{}]", thread.getName, name) - //shutdown() //TODO make it configurable - if (thread.isAlive) log.error("Thread is still alive") - else { - log.error("Thread is dead") - } - } + if (isFatalError(cause) && !settings.JvmExitOnFatalError) { + log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + + "ActorSystem [{}] tolerating and continuing.... ", thread.getName, name) + //shutdown() //TODO make it configurable + } else { + fallbackHandler.uncaughtException(thread, cause) } } } - - override def stop(actor: ActorRef): Unit = { - val path = actor.path - val guard = guardian.path - val sys = systemGuardian.path - path.parent match { - case `guard` ⇒ guardian ! StopChild(actor) - case `sys` ⇒ systemGuardian ! StopChild(actor) - case _ ⇒ actor.asInstanceOf[InternalActorRef].stop() - } } - - override def /(actorName: String): ActorPath = guardian.path / actorName - - override def /(path: Iterable[String]): ActorPath = guardian.path / path - - private lazy val _start: this.type = { - // the provider is expected to start default loggers, LocalActorRefProvider does this - provider.init(this) - this + def isFatalError(e: Throwable): Boolean = { + e match { + case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => + false + case _ => + true + } } - - override def start(): this.type = _start - - override def toString: String = lookupRoot.path.root.address.toString - } -- cgit v1.2.3 From 09e8be9a6225203337a01e618851e807a1482603 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 3 Dec 2013 11:27:45 +0530 Subject: Made running SparkActorSystem specific to executors only. --- .../apache/spark/executor/CoarseGrainedExecutorBackend.scala | 3 ++- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 11 +++++++++-- 2 files changed, 11 insertions(+), 3 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index dcb12bed4e..406e015f08 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -97,7 +97,8 @@ private[spark] object CoarseGrainedExecutorBackend { // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor // before getting started with all our system properties, etc - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, + useSparkAS = true) // set it val sparkHostPort = hostname + ":" + boundPort System.setProperty("spark.hostPort", sparkHostPort) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 407e9ffe90..f3e2644a58 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -35,7 +35,9 @@ private[spark] object AkkaUtils { * Note: the `name` parameter is important, as even if a client sends a message to right * host + port, if the system name is incorrect, Akka will drop the message. */ - def createActorSystem(name: String, host: String, port: Int): (ActorSystem, Int) = { + def createActorSystem(name: String, host: String, port: Int, + useSparkAS: Boolean = false): (ActorSystem, Int) = { + val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt @@ -70,7 +72,12 @@ private[spark] object AkkaUtils { |akka.remote.log-remote-lifecycle-events = $lifecycleEvents """.stripMargin) - val actorSystem = SparkActorSystem(name, akkaConf) + val actorSystem = if (useSparkAS) { + SparkActorSystem(name, akkaConf) + } + else { + ActorSystem(name, akkaConf) + } val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.getDefaultAddress.port.get -- cgit v1.2.3 From 51458ab4a16a2d365f5de756d2fac942b766feca Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Mon, 11 Nov 2013 16:06:12 -0800 Subject: Added stageId <--> jobId mapping in DAGScheduler ...and make sure that DAGScheduler data structures are cleaned up on job completion. Initial effort and discussion at https://github.com/mesos/spark/pull/842 --- .../scala/org/apache/spark/MapOutputTracker.scala | 8 +- .../org/apache/spark/scheduler/DAGScheduler.scala | 277 ++++++++++++++++----- .../apache/spark/scheduler/DAGSchedulerEvent.scala | 5 +- .../org/apache/spark/scheduler/SparkListener.scala | 2 +- .../spark/scheduler/cluster/ClusterScheduler.scala | 4 +- .../scheduler/cluster/ClusterTaskSetManager.scala | 2 +- .../spark/scheduler/local/LocalScheduler.scala | 27 +- .../org/apache/spark/JobCancellationSuite.scala | 4 +- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 45 +++- 9 files changed, 286 insertions(+), 88 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 5e465fa22c..b4d0b7017c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -244,12 +244,12 @@ private[spark] class MapOutputTrackerMaster extends MapOutputTracker { case Some(bytes) => return bytes case None => - statuses = mapStatuses(shuffleId) + statuses = mapStatuses.getOrElse(shuffleId, Array[MapStatus]()) 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 + // out a snapshot of the locations as "statuses"; let's serialize and return that val bytes = MapOutputTracker.serializeMapStatuses(statuses) logInfo("Size of output statuses for shuffle %d is %d bytes".format(shuffleId, bytes.length)) // Add them into the table only if the epoch hasn't changed while we were working @@ -274,6 +274,10 @@ private[spark] class MapOutputTrackerMaster extends MapOutputTracker { override def updateEpoch(newEpoch: Long) { // This might be called on the MapOutputTrackerMaster if we're running in local mode. } + + def has(shuffleId: Int): Boolean = { + cachedSerializedStatuses.get(shuffleId).isDefined || mapStatuses.contains(shuffleId) + } } private[spark] object MapOutputTracker { 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 a785a16a36..10417b9343 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -121,9 +121,13 @@ class DAGScheduler( private val nextStageId = new AtomicInteger(0) - private val stageIdToStage = new TimeStampedHashMap[Int, Stage] + private[scheduler] val jobIdToStageIds = new TimeStampedHashMap[Int, HashSet[Int]] - private val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] + private[scheduler] val stageIdToJobIds = new TimeStampedHashMap[Int, HashSet[Int]] + + private[scheduler] val stageIdToStage = new TimeStampedHashMap[Int, Stage] + + private[scheduler] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] @@ -232,7 +236,7 @@ class DAGScheduler( shuffleToMapStage.get(shuffleDep.shuffleId) match { case Some(stage) => stage case None => - val stage = newStage(shuffleDep.rdd, shuffleDep.rdd.partitions.size, Some(shuffleDep), jobId) + val stage = newOrUsedStage(shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId) shuffleToMapStage(shuffleDep.shuffleId) = stage stage } @@ -241,7 +245,8 @@ 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 - * associated with the provided jobId. + * associated with the provided jobId.. Shuffle map stages, whose shuffleId may have previously + * been registered in the MapOutputTracker, should be (re)-created using newOrUsedStage. */ private def newStage( rdd: RDD[_], @@ -251,20 +256,44 @@ class DAGScheduler( 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 - logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")") - mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size) - } val id = nextStageId.getAndIncrement() val stage = new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite) stageIdToStage(id) = stage + registerJobIdWithStages(jobId, stage) stageToInfos(stage) = new StageInfo(stage) stage } + /** + * Create a shuffle map Stage for the given RDD. The stage will also be associated with the + * provided jobId. If a stage for the shuffleId existed previously so that the shuffleId is + * present in the MapOutputTracker, then the number and location of available outputs are + * recovered from the MapOutputTracker + */ + private def newOrUsedStage( + rdd: RDD[_], + numTasks: Int, + shuffleDep: ShuffleDependency[_,_], + jobId: Int, + callSite: Option[String] = None) + : Stage = + { + val stage = newStage(rdd, numTasks, Some(shuffleDep), jobId, callSite) + if (mapOutputTracker.has(shuffleDep.shuffleId)) { + val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId) + val locs = MapOutputTracker.deserializeMapStatuses(serLocs) + for (i <- 0 until locs.size) stage.outputLocs(i) = List(locs(i)) + stage.numAvailableOutputs = locs.size + } else { + // 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 + logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")") + mapOutputTracker.registerShuffle(shuffleDep.shuffleId, rdd.partitions.size) + } + stage + } + /** * Get or create the list of parent stages for a given RDD. The stages will be assigned the * provided jobId if they haven't already been created with a lower jobId. @@ -316,6 +345,91 @@ class DAGScheduler( missing.toList } + /** + * Registers the given jobId among the jobs that need the given stage and + * all of that stage's ancestors. + */ + private def registerJobIdWithStages(jobId: Int, stage: Stage) { + def registerJobIdWithStageList(stages: List[Stage]) { + if (!stages.isEmpty) { + val s = stages.head + stageIdToJobIds.getOrElseUpdate(s.id, new HashSet[Int]()) += jobId + val parents = getParentStages(s.rdd, jobId) + val parentsWithoutThisJobId = parents.filter(p => !stageIdToJobIds.get(p.id).exists(_.contains(jobId))) + registerJobIdWithStageList(parentsWithoutThisJobId ++ stages.tail) + } + } + registerJobIdWithStageList(List(stage)) + } + + private def jobIdToStageIdsAdd(jobId: Int) { + val stageSet = jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) + stageIdToJobIds.foreach { case (stageId, jobSet) => + if (jobSet.contains(jobId)) { + stageSet += stageId + } + } + } + + // Removes job and applies p to any stages that aren't needed by any other jobs + private def forIndependentStagesOfRemovedJob(jobId: Int)(p: Int => Unit) { + val registeredStages = jobIdToStageIds(jobId) + if (registeredStages.isEmpty) { + logError("No stages registered for job " + jobId) + } else { + stageIdToJobIds.filterKeys(stageId => registeredStages.contains(stageId)).foreach { + case (stageId, jobSet) => + if (!jobSet.contains(jobId)) { + logError("Job %d not registered for stage %d even though that stage was registered for the job" + .format(jobId, stageId)) + } else { + jobSet -= jobId + if ((jobSet - jobId).isEmpty) { // no other job needs this stage + p(stageId) + } + } + } + } + } + + private def removeStage(stageId: Int) { + // data structures based on Stage + stageIdToStage.get(stageId).foreach { s => + if (running.contains(s)) { + logDebug("Removing running stage %d".format(stageId)) + running -= s + } + stageToInfos -= s + shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach(shuffleToMapStage.remove(_)) + if (pendingTasks.contains(s) && !pendingTasks(s).isEmpty) { + logDebug("Removing pending status for stage %d".format(stageId)) + } + pendingTasks -= s + if (waiting.contains(s)) { + logDebug("Removing stage %d from waiting set.".format(stageId)) + waiting -= s + } + if (failed.contains(s)) { + logDebug("Removing stage %d from failed set.".format(stageId)) + failed -= s + } + } + // data structures based on StageId + stageIdToStage -= stageId + stageIdToJobIds -= stageId + + logDebug("After removal of stage %d, remaining stages = %d".format(stageId, stageIdToStage.size)) + } + + private def jobIdToStageIdsRemove(jobId: Int) { + if (!jobIdToStageIds.contains(jobId)) { + logDebug("Trying to remove unregistered job " + jobId) + } else { + forIndependentStagesOfRemovedJob(jobId) { removeStage } + jobIdToStageIds -= jobId + } + } + /** * Submit a job to the job scheduler and get a JobWaiter object back. The JobWaiter object * can be used to block until the the job finishes executing or can be used to cancel the job. @@ -435,35 +549,33 @@ class DAGScheduler( // Compute very short actions like first() or take() with no parent stages locally. runLocally(job) } else { - listenerBus.post(SparkListenerJobStart(job, properties)) idToActiveJob(jobId) = job activeJobs += job resultStageToJob(finalStage) = job + jobIdToStageIdsAdd(jobId) + listenerBus.post(SparkListenerJobStart(job, jobIdToStageIds(jobId).toArray, properties)) submitStage(finalStage) } case JobCancelled(jobId) => - // Cancel a job: find all the running stages that are linked to this job, and cancel them. - running.filter(_.jobId == jobId).foreach { stage => - taskSched.cancelTasks(stage.id) - } + handleJobCancellation(jobId) + idToActiveJob.get(jobId).foreach(job => activeJobs -= job) + idToActiveJob -= jobId case JobGroupCancelled(groupId) => // Cancel all jobs belonging to this job group. // First finds all active jobs with this group id, and then kill stages for them. - val jobIds = activeJobs.filter(groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) - .map(_.jobId) - if (!jobIds.isEmpty) { - running.filter(stage => jobIds.contains(stage.jobId)).foreach { stage => - taskSched.cancelTasks(stage.id) - } - } + val activeInGroup = activeJobs.filter(groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) + val jobIds = activeInGroup.map(_.jobId) + jobIds.foreach { handleJobCancellation } + activeJobs -- activeInGroup + idToActiveJob -- jobIds case AllJobsCancelled => // Cancel all running jobs. - running.foreach { stage => - taskSched.cancelTasks(stage.id) - } + running.map(_.jobId).foreach { handleJobCancellation } + activeJobs.clear() + idToActiveJob.clear() case ExecutorGained(execId, host) => handleExecutorGained(execId, host) @@ -493,8 +605,13 @@ class DAGScheduler( listenerBus.post(SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics)) handleTaskCompletion(completion) + case LocalJobCompleted(stage) => + stageIdToJobIds -= stage.id // clean up data structures that were populated for a local job, + stageIdToStage -= stage.id // but that won't get cleaned up via the normal paths through + stageToInfos -= stage // completion events or stage abort + case TaskSetFailed(taskSet, reason) => - abortStage(stageIdToStage(taskSet.stageId), reason) + stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } case ResubmitFailedStages => if (failed.size > 0) { @@ -576,30 +693,52 @@ class DAGScheduler( } catch { case e: Exception => job.listener.jobFailed(e) + } finally { + eventQueue.put(LocalJobCompleted(job.finalStage)) + } + } + + /** Finds the earliest-created active job that needs the stage */ + // TODO: Probably should actually find among the active jobs that need this + // stage the one with the highest priority (highest-priority pool, earliest created). + // That should take care of at least part of the priority inversion problem with + // cross-job dependencies. + private def activeJobForStage(stage: Stage): Option[Int] = { + if (stageIdToJobIds.contains(stage.id)) { + val jobsThatUseStage: Array[Int] = stageIdToJobIds(stage.id).toArray.sorted + jobsThatUseStage.find(idToActiveJob.contains(_)) + } else { + None } } /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { - logDebug("submitStage(" + stage + ")") - if (!waiting(stage) && !running(stage) && !failed(stage)) { - val missing = getMissingParentStages(stage).sortBy(_.id) - logDebug("missing: " + missing) - if (missing == Nil) { - logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents") - submitMissingTasks(stage) - running += stage - } else { - for (parent <- missing) { - submitStage(parent) + val jobId = activeJobForStage(stage) + if (jobId.isDefined) { + logDebug("submitStage(" + stage + ")") + if (!waiting(stage) && !running(stage) && !failed(stage)) { + val missing = getMissingParentStages(stage).sortBy(_.id) + logDebug("missing: " + missing) + if (missing == Nil) { + logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents") + submitMissingTasks(stage, jobId.get) + running += stage + } else { + for (parent <- missing) { + submitStage(parent) + } + waiting += stage } - waiting += stage } + } else { + abortStage(stage, "No active job for stage " + stage.id) } } + /** Called when stage's parents are available and we can now do its task. */ - private def submitMissingTasks(stage: Stage) { + private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") // Get our pending tasks and remember them in our pendingTasks entry val myPending = pendingTasks.getOrElseUpdate(stage, new HashSet) @@ -620,7 +759,7 @@ class DAGScheduler( } } - val properties = if (idToActiveJob.contains(stage.jobId)) { + val properties = if (idToActiveJob.contains(jobId)) { idToActiveJob(stage.jobId).properties } else { //this stage will be assigned to "default" pool @@ -703,6 +842,7 @@ class DAGScheduler( resultStageToJob -= stage markStageAsFinished(stage) listenerBus.post(SparkListenerJobEnd(job, JobSucceeded)) + jobIdToStageIdsRemove(job.jobId) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -738,7 +878,7 @@ class DAGScheduler( changeEpoch = true) } clearCacheLocs() - if (stage.outputLocs.count(_ == Nil) != 0) { + if (stage.outputLocs.exists(_ == Nil)) { // Some tasks had failed; let's resubmit this stage // TODO: Lower-level scheduler should also deal with this logInfo("Resubmitting " + stage + " (" + stage.name + @@ -755,9 +895,12 @@ class DAGScheduler( } waiting --= newlyRunnable running ++= newlyRunnable - for (stage <- newlyRunnable.sortBy(_.id)) { + for { + stage <- newlyRunnable.sortBy(_.id) + jobId <- activeJobForStage(stage) + } { logInfo("Submitting " + stage + " (" + stage.rdd + "), which is now runnable") - submitMissingTasks(stage) + submitMissingTasks(stage, jobId) } } } @@ -841,11 +984,31 @@ class DAGScheduler( } } + private def handleJobCancellation(jobId: Int) { + if (!jobIdToStageIds.contains(jobId)) { + logDebug("Trying to cancel unregistered job " + jobId) + } else { + forIndependentStagesOfRemovedJob(jobId) { stageId => + taskSched.cancelTasks(stageId) + removeStage(stageId) + } + val error = new SparkException("Job %d cancelled".format(jobId)) + val job = idToActiveJob(jobId) + job.listener.jobFailed(error) + listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(job.finalStage)))) + jobIdToStageIds -= jobId + } + } + /** * Aborts all jobs depending on a particular Stage. This is called in response to a task set * being canceled by the TaskScheduler. Use taskSetFailed() to inject this event from outside. */ private def abortStage(failedStage: Stage, reason: String) { + if (!stageIdToStage.contains(failedStage.id)) { + // Skip all the actions if the stage has been removed. + return + } val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq stageToInfos(failedStage).completionTime = Some(System.currentTimeMillis()) for (resultStage <- dependentStages) { @@ -853,6 +1016,7 @@ class DAGScheduler( val error = new SparkException("Job aborted: " + reason) job.listener.jobFailed(error) listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage)))) + jobIdToStageIdsRemove(job.jobId) idToActiveJob -= resultStage.jobId activeJobs -= job resultStageToJob -= resultStage @@ -926,21 +1090,18 @@ class DAGScheduler( } private def cleanup(cleanupTime: Long) { - var sizeBefore = stageIdToStage.size - stageIdToStage.clearOldValues(cleanupTime) - logInfo("stageIdToStage " + sizeBefore + " --> " + stageIdToStage.size) - - sizeBefore = shuffleToMapStage.size - shuffleToMapStage.clearOldValues(cleanupTime) - logInfo("shuffleToMapStage " + sizeBefore + " --> " + shuffleToMapStage.size) - - sizeBefore = pendingTasks.size - pendingTasks.clearOldValues(cleanupTime) - logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size) - - sizeBefore = stageToInfos.size - stageToInfos.clearOldValues(cleanupTime) - logInfo("stageToInfos " + sizeBefore + " --> " + stageToInfos.size) + Map( + "stageIdToStage" -> stageIdToStage, + "shuffleToMapStage" -> shuffleToMapStage, + "pendingTasks" -> pendingTasks, + "stageToInfos" -> stageToInfos, + "jobIdToStageIds" -> jobIdToStageIds, + "stageIdToJobIds" -> stageIdToJobIds). + foreach { case(s, t) => { + val sizeBefore = t.size + t.clearOldValues(cleanupTime) + logInfo("%s %d --> %d".format(s, sizeBefore, t.size)) + }} } def stop() { 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 5353cd24dc..bf8dfb5ac7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -65,8 +65,9 @@ private[scheduler] case class CompletionEvent( taskMetrics: TaskMetrics) extends DAGSchedulerEvent -private[scheduler] -case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent +private[scheduler] case class LocalJobCompleted(stage: Stage) extends DAGSchedulerEvent + +private[scheduler] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent private[scheduler] case class ExecutorLost(execId: String) extends DAGSchedulerEvent 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 a35081f7b1..3841b5616d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -37,7 +37,7 @@ case class SparkListenerTaskGettingResult( case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo, taskMetrics: TaskMetrics) extends SparkListenerEvents -case class SparkListenerJobStart(job: ActiveJob, properties: Properties = null) +case class SparkListenerJobStart(job: ActiveJob, stageIds: Array[Int], properties: Properties = null) extends SparkListenerEvents case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala index c1e65a3c48..bd0a39b4d2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala @@ -173,7 +173,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) backend.killTask(tid, execId) } } - tsm.error("Stage %d was cancelled".format(stageId)) + logInfo("Stage %d was cancelled".format(stageId)) + tsm.removeAllRunningTasks() + taskSetFinished(tsm) } } 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 8884ea85a3..94961790df 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 @@ -574,7 +574,7 @@ private[spark] class ClusterTaskSetManager( runningTasks = runningTasksSet.size } - private def removeAllRunningTasks() { + private[cluster] def removeAllRunningTasks() { val numRunningTasks = runningTasksSet.size runningTasksSet.clear() if (parent != null) { 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 5af51164f7..01e95162c0 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 @@ -144,7 +144,8 @@ private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val localActor ! KillTask(tid) } } - tsm.error("Stage %d was cancelled".format(stageId)) + logInfo("Stage %d was cancelled".format(stageId)) + taskSetFinished(tsm) } } @@ -192,17 +193,19 @@ private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val synchronized { taskIdToTaskSetId.get(taskId) match { case Some(taskSetId) => - val taskSetManager = activeTaskSets(taskSetId) - taskSetTaskIds(taskSetId) -= taskId - - state match { - case TaskState.FINISHED => - taskSetManager.taskEnded(taskId, state, serializedData) - case TaskState.FAILED => - taskSetManager.taskFailed(taskId, state, serializedData) - case TaskState.KILLED => - taskSetManager.error("Task %d was killed".format(taskId)) - case _ => {} + val taskSetManager = activeTaskSets.get(taskSetId) + taskSetManager.foreach { tsm => + taskSetTaskIds(taskSetId) -= taskId + + state match { + case TaskState.FINISHED => + tsm.taskEnded(taskId, state, serializedData) + case TaskState.FAILED => + tsm.taskFailed(taskId, state, serializedData) + case TaskState.KILLED => + tsm.error("Task %d was killed".format(taskId)) + case _ => {} + } } case None => logInfo("Ignoring update from TID " + taskId + " because its task set is gone") diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index d8a0e983b2..1121e06e2e 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -114,7 +114,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf // Once A is cancelled, job B should finish fairly quickly. assert(jobB.get() === 100) } - +/* test("two jobs sharing the same stage") { // sem1: make sure cancel is issued after some tasks are launched // sem2: make sure the first stage is not finished until cancel is issued @@ -148,7 +148,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf intercept[SparkException] { f1.get() } intercept[SparkException] { f2.get() } } - + */ def testCount() { // Cancel before launching any tasks { 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 a4d41ebbff..8ce8c68af3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -206,6 +206,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont submit(rdd, Array(0)) complete(taskSets(0), List((Success, 42))) assert(results === Map(0 -> 42)) + assertDataStructuresEmpty } test("local job") { @@ -218,7 +219,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont } val jobId = scheduler.nextJobId.getAndIncrement() runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, listener)) + assert(scheduler.stageToInfos.size === 1) + runEvent(LocalJobCompleted(scheduler.stageToInfos.keys.head)) assert(results === Map(0 -> 42)) + assertDataStructuresEmpty } test("run trivial job w/ dependency") { @@ -227,6 +231,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont submit(finalRdd, Array(0)) complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) + assertDataStructuresEmpty } test("cache location preferences w/ dependency") { @@ -239,12 +244,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assertLocations(taskSet, Seq(Seq("hostA", "hostB"))) complete(taskSet, Seq((Success, 42))) assert(results === Map(0 -> 42)) + assertDataStructuresEmpty } test("trivial job failure") { submit(makeRdd(1, Nil), Array(0)) failed(taskSets(0), "some failure") assert(failure.getMessage === "Job aborted: some failure") + assertDataStructuresEmpty } test("run trivial shuffle") { @@ -260,6 +267,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) complete(taskSets(1), Seq((Success, 42))) assert(results === Map(0 -> 42)) + assertDataStructuresEmpty } test("run trivial shuffle with fetch failure") { @@ -285,6 +293,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === Array("hostA", "hostB")) complete(taskSets(3), Seq((Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) + assertDataStructuresEmpty } test("ignore late map task completions") { @@ -313,6 +322,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) complete(taskSets(1), Seq((Success, 42), (Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) + assertDataStructuresEmpty } test("run trivial shuffle with out-of-band failure and retry") { @@ -329,15 +339,16 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) - // have hostC complete the resubmitted task - complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === - Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) - complete(taskSets(2), Seq((Success, 42))) - assert(results === Map(0 -> 42)) - } - - test("recursive shuffle failures") { + // have hostC complete the resubmitted task + complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) + complete(taskSets(2), Seq((Success, 42))) + assert(results === Map(0 -> 42)) + assertDataStructuresEmpty + } + + test("recursive shuffle failures") { val shuffleOneRdd = makeRdd(2, Nil) val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) @@ -363,6 +374,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont complete(taskSets(4), Seq((Success, makeMapStatus("hostA", 1)))) complete(taskSets(5), Seq((Success, 42))) assert(results === Map(0 -> 42)) + assertDataStructuresEmpty } test("cached post-shuffle") { @@ -394,6 +406,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont complete(taskSets(3), Seq((Success, makeMapStatus("hostD", 1)))) complete(taskSets(4), Seq((Success, 42))) assert(results === Map(0 -> 42)) + assertDataStructuresEmpty } /** @@ -413,4 +426,18 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont private def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345, 0) + private def assertDataStructuresEmpty = { + assert(scheduler.pendingTasks.isEmpty) + assert(scheduler.activeJobs.isEmpty) + assert(scheduler.failed.isEmpty) + assert(scheduler.idToActiveJob.isEmpty) + assert(scheduler.jobIdToStageIds.isEmpty) + assert(scheduler.stageIdToJobIds.isEmpty) + assert(scheduler.stageIdToStage.isEmpty) + assert(scheduler.stageToInfos.isEmpty) + assert(scheduler.resultStageToJob.isEmpty) + assert(scheduler.running.isEmpty) + assert(scheduler.shuffleToMapStage.isEmpty) + assert(scheduler.waiting.isEmpty) + } } -- cgit v1.2.3 From 6f8359b5ad6c069c6105631a6c74e225b866cfce Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 19 Nov 2013 10:16:48 -0800 Subject: Actor instead of eventQueue for LocalJobCompleted --- core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core/src') 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 10417b9343..ad436f854c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -694,7 +694,7 @@ class DAGScheduler( case e: Exception => job.listener.jobFailed(e) } finally { - eventQueue.put(LocalJobCompleted(job.finalStage)) + eventProcessActor ! LocalJobCompleted(job.finalStage) } } -- cgit v1.2.3 From 982797dcbafa4c1149ad354b0c5a07e3f74fe005 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 19 Nov 2013 16:59:42 -0800 Subject: Fixed intended side-effects --- core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') 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 ad436f854c..bf5827d011 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -568,8 +568,8 @@ class DAGScheduler( val activeInGroup = activeJobs.filter(groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) val jobIds = activeInGroup.map(_.jobId) jobIds.foreach { handleJobCancellation } - activeJobs -- activeInGroup - idToActiveJob -- jobIds + activeJobs --= activeInGroup + idToActiveJob --= jobIds case AllJobsCancelled => // Cancel all running jobs. -- cgit v1.2.3 From 94087c463b41a92a9462b954f1f6452614569fe5 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 20 Nov 2013 15:47:30 -0800 Subject: Removed redundant residual re: reverted refactoring. --- core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core/src') 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 bf5827d011..be46f74f7c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -384,7 +384,7 @@ class DAGScheduler( .format(jobId, stageId)) } else { jobSet -= jobId - if ((jobSet - jobId).isEmpty) { // no other job needs this stage + if (jobSet.isEmpty) { // no other job needs this stage p(stageId) } } -- cgit v1.2.3 From 205566e56e2891245b2d7820bfb3629945a2dcd9 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 20 Nov 2013 14:49:09 -0800 Subject: Improved comment --- core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) (limited to 'core/src') 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 be46f74f7c..6f9d4d52a4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -243,10 +243,9 @@ 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 - * associated with the provided jobId.. Shuffle map stages, whose shuffleId may have previously - * been registered in the MapOutputTracker, should be (re)-created using newOrUsedStage. + * Create a Stage -- either directly for use as a result stage, or as part of the (re)-creation + * of a shuffle map stage in newOrUsedStage. The stage will be associated with the provided + * jobId. Production of shuffle map stages should always use newOrUsedStage, not newStage directly. */ private def newStage( rdd: RDD[_], -- cgit v1.2.3 From 686a420ddc33407050d9019711cbe801fc352fa3 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 22 Nov 2013 10:20:09 -0800 Subject: Refactoring to make job removal, stage removal, task cancellation clearer --- .../org/apache/spark/scheduler/DAGScheduler.scala | 76 +++++++++++----------- 1 file changed, 39 insertions(+), 37 deletions(-) (limited to 'core/src') 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 6f9d4d52a4..b8b3ac0b43 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -370,9 +370,11 @@ class DAGScheduler( } } - // Removes job and applies p to any stages that aren't needed by any other jobs - private def forIndependentStagesOfRemovedJob(jobId: Int)(p: Int => Unit) { + // Removes job and any stages that are not needed by any other job. Returns the set of ids for stages that + // were removed and whose associated tasks may need to be cancelled. + private def removeJobAndIndependentStages(jobId: Int): Set[Int] = { val registeredStages = jobIdToStageIds(jobId) + val independentStages = new HashSet[Int]() if (registeredStages.isEmpty) { logError("No stages registered for job " + jobId) } else { @@ -382,49 +384,51 @@ class DAGScheduler( logError("Job %d not registered for stage %d even though that stage was registered for the job" .format(jobId, stageId)) } else { + def removeStage(stageId: Int) { + // data structures based on Stage + stageIdToStage.get(stageId).foreach { s => + if (running.contains(s)) { + logDebug("Removing running stage %d".format(stageId)) + running -= s + } + stageToInfos -= s + shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach(shuffleToMapStage.remove) + if (pendingTasks.contains(s) && !pendingTasks(s).isEmpty) { + logDebug("Removing pending status for stage %d".format(stageId)) + } + pendingTasks -= s + if (waiting.contains(s)) { + logDebug("Removing stage %d from waiting set.".format(stageId)) + waiting -= s + } + if (failed.contains(s)) { + logDebug("Removing stage %d from failed set.".format(stageId)) + failed -= s + } + } + // data structures based on StageId + stageIdToStage -= stageId + stageIdToJobIds -= stageId + + logDebug("After removal of stage %d, remaining stages = %d".format(stageId, stageIdToStage.size)) + } + jobSet -= jobId if (jobSet.isEmpty) { // no other job needs this stage - p(stageId) + independentStages += stageId + removeStage(stageId) } } } } - } - - private def removeStage(stageId: Int) { - // data structures based on Stage - stageIdToStage.get(stageId).foreach { s => - if (running.contains(s)) { - logDebug("Removing running stage %d".format(stageId)) - running -= s - } - stageToInfos -= s - shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach(shuffleToMapStage.remove(_)) - if (pendingTasks.contains(s) && !pendingTasks(s).isEmpty) { - logDebug("Removing pending status for stage %d".format(stageId)) - } - pendingTasks -= s - if (waiting.contains(s)) { - logDebug("Removing stage %d from waiting set.".format(stageId)) - waiting -= s - } - if (failed.contains(s)) { - logDebug("Removing stage %d from failed set.".format(stageId)) - failed -= s - } - } - // data structures based on StageId - stageIdToStage -= stageId - stageIdToJobIds -= stageId - - logDebug("After removal of stage %d, remaining stages = %d".format(stageId, stageIdToStage.size)) + independentStages.toSet } private def jobIdToStageIdsRemove(jobId: Int) { if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to remove unregistered job " + jobId) } else { - forIndependentStagesOfRemovedJob(jobId) { removeStage } + removeJobAndIndependentStages(jobId) jobIdToStageIds -= jobId } } @@ -987,10 +991,8 @@ class DAGScheduler( if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - forIndependentStagesOfRemovedJob(jobId) { stageId => - taskSched.cancelTasks(stageId) - removeStage(stageId) - } + val independentStages = removeJobAndIndependentStages(jobId) + independentStages.foreach { taskSched.cancelTasks } val error = new SparkException("Job %d cancelled".format(jobId)) val job = idToActiveJob(jobId) job.listener.jobFailed(error) -- cgit v1.2.3 From 27c45e523620d801d547f167a5a33d71ee3af7b5 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 22 Nov 2013 11:14:39 -0800 Subject: Cleaned up job cancellation handling --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) (limited to 'core/src') 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 b8b3ac0b43..aeac14ad7b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -371,7 +371,7 @@ class DAGScheduler( } // Removes job and any stages that are not needed by any other job. Returns the set of ids for stages that - // were removed and whose associated tasks may need to be cancelled. + // were removed. The associated tasks for those stages need to be cancelled if we got here via job cancellation. private def removeJobAndIndependentStages(jobId: Int): Set[Int] = { val registeredStages = jobIdToStageIds(jobId) val independentStages = new HashSet[Int]() @@ -562,8 +562,6 @@ class DAGScheduler( case JobCancelled(jobId) => handleJobCancellation(jobId) - idToActiveJob.get(jobId).foreach(job => activeJobs -= job) - idToActiveJob -= jobId case JobGroupCancelled(groupId) => // Cancel all jobs belonging to this job group. @@ -571,14 +569,12 @@ class DAGScheduler( val activeInGroup = activeJobs.filter(groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) val jobIds = activeInGroup.map(_.jobId) jobIds.foreach { handleJobCancellation } - activeJobs --= activeInGroup - idToActiveJob --= jobIds case AllJobsCancelled => // Cancel all running jobs. running.map(_.jobId).foreach { handleJobCancellation } - activeJobs.clear() - idToActiveJob.clear() + activeJobs.clear() // These should already be empty by this point, + idToActiveJob.clear() // but just in case we lost track of some jobs... case ExecutorGained(execId, host) => handleExecutorGained(execId, host) @@ -998,6 +994,8 @@ class DAGScheduler( job.listener.jobFailed(error) listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(job.finalStage)))) jobIdToStageIds -= jobId + activeJobs -= job + idToActiveJob -= jobId } } -- cgit v1.2.3 From 9ae2d094a967782e3f5a624dd854059a40430ee6 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 22 Nov 2013 13:14:26 -0800 Subject: Tightly couple stageIdToJobIds and jobIdToStageIds --- .../org/apache/spark/scheduler/DAGScheduler.scala | 29 +++++++++------------- 1 file changed, 12 insertions(+), 17 deletions(-) (limited to 'core/src') 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 aeac14ad7b..01c5133e6e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -259,7 +259,7 @@ class DAGScheduler( val stage = new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite) stageIdToStage(id) = stage - registerJobIdWithStages(jobId, stage) + updateJobIdStageIdMaps(jobId, stage) stageToInfos(stage) = new StageInfo(stage) stage } @@ -348,30 +348,24 @@ class DAGScheduler( * Registers the given jobId among the jobs that need the given stage and * all of that stage's ancestors. */ - private def registerJobIdWithStages(jobId: Int, stage: Stage) { - def registerJobIdWithStageList(stages: List[Stage]) { + private def updateJobIdStageIdMaps(jobId: Int, stage: Stage) { + def updateJobIdStageIdMapsList(stages: List[Stage]) { if (!stages.isEmpty) { val s = stages.head stageIdToJobIds.getOrElseUpdate(s.id, new HashSet[Int]()) += jobId + jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) += s.id val parents = getParentStages(s.rdd, jobId) val parentsWithoutThisJobId = parents.filter(p => !stageIdToJobIds.get(p.id).exists(_.contains(jobId))) - registerJobIdWithStageList(parentsWithoutThisJobId ++ stages.tail) + updateJobIdStageIdMapsList(parentsWithoutThisJobId ++ stages.tail) } } - registerJobIdWithStageList(List(stage)) + updateJobIdStageIdMapsList(List(stage)) } - private def jobIdToStageIdsAdd(jobId: Int) { - val stageSet = jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) - stageIdToJobIds.foreach { case (stageId, jobSet) => - if (jobSet.contains(jobId)) { - stageSet += stageId - } - } - } - - // Removes job and any stages that are not needed by any other job. Returns the set of ids for stages that - // were removed. The associated tasks for those stages need to be cancelled if we got here via job cancellation. + /** + * Removes job and any stages that are not needed by any other job. Returns the set of ids for stages that + * were removed. The associated tasks for those stages need to be cancelled if we got here via job cancellation. + */ private def removeJobAndIndependentStages(jobId: Int): Set[Int] = { val registeredStages = jobIdToStageIds(jobId) val independentStages = new HashSet[Int]() @@ -555,7 +549,6 @@ class DAGScheduler( idToActiveJob(jobId) = job activeJobs += job resultStageToJob(finalStage) = job - jobIdToStageIdsAdd(jobId) listenerBus.post(SparkListenerJobStart(job, jobIdToStageIds(jobId).toArray, properties)) submitStage(finalStage) } @@ -605,9 +598,11 @@ class DAGScheduler( handleTaskCompletion(completion) case LocalJobCompleted(stage) => + val jobId = stageIdToJobIds(stage.id).head stageIdToJobIds -= stage.id // clean up data structures that were populated for a local job, stageIdToStage -= stage.id // but that won't get cleaned up via the normal paths through stageToInfos -= stage // completion events or stage abort + jobIdToStageIds -= jobId case TaskSetFailed(taskSet, reason) => stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } -- cgit v1.2.3 From c9fcd909d0f86b08935a132409888b30e989bca4 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Sun, 24 Nov 2013 17:49:14 -0800 Subject: Local jobs post SparkListenerJobEnd, and DAGScheduler data structure cleanup always occurs before any posting of SparkListenerJobEnd. --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 17 ++++++++++------- .../org/apache/spark/scheduler/DAGSchedulerEvent.scala | 2 +- 2 files changed, 11 insertions(+), 8 deletions(-) (limited to 'core/src') 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 01c5133e6e..b371a2412f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -597,12 +597,13 @@ class DAGScheduler( listenerBus.post(SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics)) handleTaskCompletion(completion) - case LocalJobCompleted(stage) => - val jobId = stageIdToJobIds(stage.id).head + case LocalJobCompleted(job, result) => + val stage = job.finalStage stageIdToJobIds -= stage.id // clean up data structures that were populated for a local job, stageIdToStage -= stage.id // but that won't get cleaned up via the normal paths through stageToInfos -= stage // completion events or stage abort - jobIdToStageIds -= jobId + jobIdToStageIds -= job.jobId + listenerBus.post(SparkListenerJobEnd(job, result)) case TaskSetFailed(taskSet, reason) => stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } @@ -672,6 +673,7 @@ class DAGScheduler( // Broken out for easier testing in DAGSchedulerSuite. protected def runLocallyWithinThread(job: ActiveJob) { + var jobResult: JobResult = JobSucceeded try { SparkEnv.set(env) val rdd = job.finalStage.rdd @@ -686,9 +688,10 @@ class DAGScheduler( } } catch { case e: Exception => + jobResult = JobFailed(e, Some(job.finalStage)) job.listener.jobFailed(e) } finally { - eventProcessActor ! LocalJobCompleted(job.finalStage) + eventProcessActor ! LocalJobCompleted(job, jobResult) } } @@ -835,8 +838,8 @@ class DAGScheduler( activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) - listenerBus.post(SparkListenerJobEnd(job, JobSucceeded)) jobIdToStageIdsRemove(job.jobId) + listenerBus.post(SparkListenerJobEnd(job, JobSucceeded)) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -987,10 +990,10 @@ class DAGScheduler( val error = new SparkException("Job %d cancelled".format(jobId)) val job = idToActiveJob(jobId) job.listener.jobFailed(error) - listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(job.finalStage)))) jobIdToStageIds -= jobId activeJobs -= job idToActiveJob -= jobId + listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(job.finalStage)))) } } @@ -1009,11 +1012,11 @@ class DAGScheduler( val job = resultStageToJob(resultStage) val error = new SparkException("Job aborted: " + reason) job.listener.jobFailed(error) - listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage)))) jobIdToStageIdsRemove(job.jobId) idToActiveJob -= resultStage.jobId activeJobs -= job resultStageToJob -= resultStage + listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage)))) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") 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 bf8dfb5ac7..aa496b7ac6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -65,7 +65,7 @@ private[scheduler] case class CompletionEvent( taskMetrics: TaskMetrics) extends DAGSchedulerEvent -private[scheduler] case class LocalJobCompleted(stage: Stage) extends DAGSchedulerEvent +private[scheduler] case class LocalJobCompleted(job: ActiveJob, result: JobResult) extends DAGSchedulerEvent private[scheduler] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent -- cgit v1.2.3 From f55d0b935d7c148f49b15932938e91150b64466f Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 26 Nov 2013 14:06:59 -0800 Subject: Synchronous, inline cleanup after runLocally --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 15 ++++++--------- .../org/apache/spark/scheduler/DAGSchedulerEvent.scala | 2 -- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 2 -- 3 files changed, 6 insertions(+), 13 deletions(-) (limited to 'core/src') 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 b371a2412f..b849867519 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -597,14 +597,6 @@ class DAGScheduler( listenerBus.post(SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics)) handleTaskCompletion(completion) - case LocalJobCompleted(job, result) => - val stage = job.finalStage - stageIdToJobIds -= stage.id // clean up data structures that were populated for a local job, - stageIdToStage -= stage.id // but that won't get cleaned up via the normal paths through - stageToInfos -= stage // completion events or stage abort - jobIdToStageIds -= job.jobId - listenerBus.post(SparkListenerJobEnd(job, result)) - case TaskSetFailed(taskSet, reason) => stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } @@ -691,7 +683,12 @@ class DAGScheduler( jobResult = JobFailed(e, Some(job.finalStage)) job.listener.jobFailed(e) } finally { - eventProcessActor ! LocalJobCompleted(job, jobResult) + val s = job.finalStage + stageIdToJobIds -= s.id // clean up data structures that were populated for a local job, + stageIdToStage -= s.id // but that won't get cleaned up via the normal paths through + stageToInfos -= s // completion events or stage abort + jobIdToStageIds -= job.jobId + listenerBus.post(SparkListenerJobEnd(job, jobResult)) } } 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 aa496b7ac6..add1187613 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -65,8 +65,6 @@ private[scheduler] case class CompletionEvent( taskMetrics: TaskMetrics) extends DAGSchedulerEvent -private[scheduler] case class LocalJobCompleted(job: ActiveJob, result: JobResult) extends DAGSchedulerEvent - private[scheduler] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent private[scheduler] case class ExecutorLost(execId: String) extends DAGSchedulerEvent 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 8ce8c68af3..706d84a58b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -219,8 +219,6 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont } val jobId = scheduler.nextJobId.getAndIncrement() runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, listener)) - assert(scheduler.stageToInfos.size === 1) - runEvent(LocalJobCompleted(scheduler.stageToInfos.keys.head)) assert(results === Map(0 -> 42)) assertDataStructuresEmpty } -- cgit v1.2.3 From 403234dd0d63a7e89f3304d7bb31e3675d405a13 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 26 Nov 2013 22:25:20 -0800 Subject: SparkListenerJobStart posted from local jobs --- core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 1 + 1 file changed, 1 insertion(+) (limited to 'core/src') 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 b849867519..f9cd021dd3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -544,6 +544,7 @@ class DAGScheduler( logInfo("Missing parents: " + getMissingParentStages(finalStage)) if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { // Compute very short actions like first() or take() with no parent stages locally. + listenerBus.post(SparkListenerJobStart(job, Array(), properties)) runLocally(job) } else { idToActiveJob(jobId) = job -- cgit v1.2.3 From 974a69d79c4bf64fd9f27b65c1c464d33e647e20 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 Dec 2013 11:34:38 -0800 Subject: Marked doCheckpointCalled as transient. --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 96e4841c78..893708f8f2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -940,7 +940,7 @@ abstract class RDD[T: ClassManifest]( private var storageLevel: StorageLevel = StorageLevel.NONE /** Record user function generating this RDD. */ - private[spark] val origin = Utils.formatSparkCallSite + @transient private[spark] val origin = Utils.formatSparkCallSite private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T] @@ -955,7 +955,7 @@ abstract class RDD[T: ClassManifest]( def context = sc // Avoid handling doCheckpoint multiple times to prevent excessive recursion - private var doCheckpointCalled = false + @transient private var doCheckpointCalled = false /** * Performs the checkpointing of this RDD by saving this. It is called by the DAGScheduler -- cgit v1.2.3 From 217611680d09efcf6a218179081ee71c0a8d5c12 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 4 Dec 2013 11:29:20 -0800 Subject: Add missing space after "Serialized" in StorageLevel Current code creates outputs like: scala> res0.getStorageLevel.description res2: String = Serialized1x Replicated --- core/src/main/scala/org/apache/spark/storage/StorageLevel.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 632ff047d1..b5596dffd3 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -101,7 +101,7 @@ class StorageLevel private( var result = "" result += (if (useDisk) "Disk " else "") result += (if (useMemory) "Memory " else "") - result += (if (deserialized) "Deserialized " else "Serialized") + result += (if (deserialized) "Deserialized " else "Serialized ") result += "%sx Replicated".format(replication) result } -- cgit v1.2.3 From 380b90b9b360db9cb6a4edc1312704afe11eb31d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 4 Dec 2013 14:41:48 -0800 Subject: Fix small bug in web UI and minor clean-up. There was a bug where sorting order didn't work correctly for write time metrics. I also cleaned up some earlier code that fixed the same issue for read and write bytes. --- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 29 ++++++++++------------ 1 file changed, 13 insertions(+), 16 deletions(-) (limited to 'core/src') 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 fbd822867f..baccc4281a 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 @@ -152,21 +152,18 @@ private[spark] class StagePage(parent: JobProgressUI) { else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") val gcTime = metrics.map(m => m.jvmGCTime).getOrElse(0L) - var shuffleReadSortable: String = "" - var shuffleReadReadable: String = "" - if (shuffleRead) { - shuffleReadSortable = metrics.flatMap{m => m.shuffleReadMetrics}.map{s => s.remoteBytesRead}.toString() - shuffleReadReadable = metrics.flatMap{m => m.shuffleReadMetrics}.map{s => - Utils.bytesToString(s.remoteBytesRead)}.getOrElse("") - } + val maybeShuffleRead = metrics.flatMap{m => m.shuffleReadMetrics}.map{s => s.remoteBytesRead} + val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") + val shuffleReadReadable = maybeShuffleRead.map{Utils.bytesToString(_)}.getOrElse("") - var shuffleWriteSortable: String = "" - var shuffleWriteReadable: String = "" - if (shuffleWrite) { - shuffleWriteSortable = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => s.shuffleBytesWritten}.toString() - shuffleWriteReadable = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => - Utils.bytesToString(s.shuffleBytesWritten)}.getOrElse("") - } + val maybeShuffleWrite = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => s.shuffleBytesWritten} + val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("") + val shuffleWriteReadable = maybeShuffleWrite.map{Utils.bytesToString(_)}.getOrElse("") + + val maybeWriteTime = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => s.shuffleWriteTime} + val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") + val writeTimeReadable = maybeWriteTime.map{ t => t / (1000 * 1000)}.map{ ms => + if (ms == 0) "" else parent.formatDuration(ms)}.getOrElse("") @@ -187,8 +184,8 @@ private[spark] class StagePage(parent: JobProgressUI) { }} {if (shuffleWrite) { - - + + + + + + + + + + + + + + + + + + + + + + + + + +
ReleaseVersion code
CDH 4.X.X (YARN mode)2.0.0-chd4.X.X
CDH 4.X.X2.0.0-mr1-chd4.X.X
CDH 4.X.X (YARN mode)2.0.0-cdh4.X.X
CDH 4.X.X2.0.0-mr1-cdh4.X.X
CDH 3u60.20.2-cdh3u6
CDH 3u50.20.2-cdh3u5
CDH 3u40.20.2-cdh3u4
spark.akka.pausesspark.akka.heartbeat.pauses 60 Acceptable heart beat pause in seconds for akka, tune this if you expect GC pauses or network delays (reconnections) etc. @@ -298,7 +298,7 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.interval 5 - A larger interval value in seconds reduces network overhead and a smaller value might be more informative for akka's failure detector. Tune this in combination of `spark.akka.pauses` and `spark.akka.failure-detector.threshold` if you need to. + A larger interval value in seconds reduces network overhead and a smaller value might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to.
{info.index}{metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => - parent.formatDuration(s.shuffleWriteTime / (1000 * 1000))}.getOrElse("")} + + {writeTimeReadable} {shuffleWriteReadable} -- cgit v1.2.3 From b1c6fa1584099b3a1e0615c100f10ea90b1ad2c9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 4 Dec 2013 18:39:34 -0800 Subject: Document missing configs and set shuffle consolidation to false. --- .../apache/spark/storage/ShuffleBlockManager.scala | 2 +- docs/configuration.md | 37 +++++++++++++++++++++- 2 files changed, 37 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 2f1b049ce4..e828e1d1c5 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -62,7 +62,7 @@ class ShuffleBlockManager(blockManager: BlockManager) { // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. // TODO: Remove this once the shuffle file consolidation feature is stable. val consolidateShuffleFiles = - System.getProperty("spark.shuffle.consolidateFiles", "true").toBoolean + System.getProperty("spark.shuffle.consolidateFiles", "false").toBoolean private val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 diff --git a/docs/configuration.md b/docs/configuration.md index 97183bafdb..1a3eef345c 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -327,7 +327,42 @@ Apart from these, the following properties are also available, and may be useful Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, BlockManager might take a performance hit.
spark.shuffle.consolidateFilesfalse + If set to "true", consolidates intermediate files created during a shuffle. +
spark.speculationfalse + If set to "true", performs speculative execution of tasks. This means if one or more tasks are running slowly in a stage, they will be re-launched. +
spark.speculation.interval100 + How often Spark will check for tasks to speculate, in seconds. +
spark.speculation.quantile0.75 + Percentage of tasks which must be complete before speculation is enabled for a particular stage. +
spark.speculation.multiplier1.5 + How many times slower a task is than the median to be considered for speculation. +
# Environment Variables -- cgit v1.2.3 From 75d161b35702b6790aa66fff06b07f306442f5a3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 5 Dec 2013 11:36:16 -0800 Subject: Forcing shuffle consolidation in DiskBlockManagerSuite --- .../org/apache/spark/storage/DiskBlockManagerSuite.scala | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 0b9056344c..ef4c4c0f14 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -5,9 +5,9 @@ import java.io.{FileWriter, File} import scala.collection.mutable import com.google.common.io.Files -import org.scalatest.{BeforeAndAfterEach, FunSuite} +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} -class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { +class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { val rootDir0 = Files.createTempDir() rootDir0.deleteOnExit() @@ -16,6 +16,12 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { val rootDirs = rootDir0.getName + "," + rootDir1.getName println("Created root dirs: " + rootDirs) + // This suite focuses primarily on consolidation features, + // so we coerce consolidation if not already enabled. + val consolidateProp = "spark.shuffle.consolidateFiles" + val oldConsolidate = Option(System.getProperty(consolidateProp)) + System.setProperty(consolidateProp, "true") + val shuffleBlockManager = new ShuffleBlockManager(null) { var idToSegmentMap = mutable.Map[ShuffleBlockId, FileSegment]() override def getBlockLocation(id: ShuffleBlockId) = idToSegmentMap(id) @@ -23,6 +29,10 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { var diskBlockManager: DiskBlockManager = _ + override def afterAll() { + oldConsolidate.map(c => System.setProperty(consolidateProp, c)) + } + override def beforeEach() { diskBlockManager = new DiskBlockManager(shuffleBlockManager, rootDirs) shuffleBlockManager.idToSegmentMap.clear() -- cgit v1.2.3 From aebb123fd3b4bf0d57d867f33ca0325340ee42e4 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 5 Dec 2013 17:16:44 -0800 Subject: jobWaiter.synchronized before jobWaiter.wait --- core/src/main/scala/org/apache/spark/FutureAction.scala | 2 +- core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 1ad9240cfa..c6b4ac5192 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -99,7 +99,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: override def ready(atMost: Duration)(implicit permit: CanAwait): SimpleFutureAction.this.type = { if (!atMost.isFinite()) { awaitResult() - } else { + } else jobWaiter.synchronized { val finishTime = System.currentTimeMillis() + atMost.toMillis while (!isCompleted) { val time = System.currentTimeMillis() diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index 58f238d8cf..b026f860a8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -31,6 +31,7 @@ private[spark] class JobWaiter[T]( private var finishedTasks = 0 // Is the job as a whole finished (succeeded or failed)? + @volatile private var _jobFinished = totalTasks == 0 def jobFinished = _jobFinished -- cgit v1.2.3 From 1cb259cb577bfd3385cca6bb187d7fee18bd2c24 Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Thu, 5 Dec 2013 18:50:26 -0800 Subject: Change the name of input ragument in ClusterScheduler#initialize from context to backend. The SchedulerBackend used to be called ClusterSchedulerContext so just want to make small change of the input param in the ClusterScheduler#initialize to reflect this. --- .../scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala index c1e65a3c48..f475d000bd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala @@ -100,8 +100,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) this.dagScheduler = dagScheduler } - def initialize(context: SchedulerBackend) { - backend = context + def initialize(backend: SchedulerBackend) { + this.backend = backend // temporarily set rootPool name to empty rootPool = new Pool("", schedulingMode, 0, 0) schedulableBuilder = { -- cgit v1.2.3 From 4e70480038e9654426876e8e6b2fc356b7f0c8ca Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 6 Dec 2013 12:25:32 +0530 Subject: A left over akka -> akka.tcp changes --- .../scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala | 2 +- yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index e000531a26..e8fecec4a6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -36,7 +36,7 @@ private[spark] class SimrSchedulerBackend( override def start() { super.start() - val driverUrl = "akka://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 421a83c87a..b67e068844 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -168,7 +168,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte System.setProperty("spark.driver.host", driverHost) System.setProperty("spark.driver.port", driverPort.toString) - val driverUrl = "akka://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") -- cgit v1.2.3 From ee888f6b251c4f06f2edf15267d12e42e28fd22f Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 5 Dec 2013 21:53:40 -0800 Subject: FutureAction result tests --- .../apache/spark/rdd/AsyncRDDActionsSuite.scala | 26 ++++++++++++++++++++++ 1 file changed, 26 insertions(+) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index da032b17d9..0d4c10db8e 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.rdd import java.util.concurrent.Semaphore +import scala.concurrent.{Await, TimeoutException} +import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global import org.scalatest.{BeforeAndAfterAll, FunSuite} @@ -173,4 +175,28 @@ class AsyncRDDActionsSuite extends FunSuite with BeforeAndAfterAll with Timeouts sem.acquire(2) } } + + /** + * Awaiting FutureAction results + */ + test("FutureAction result, infinite wait") { + val f = sc.parallelize(1 to 100, 4) + .countAsync() + assert(Await.result(f, Duration.Inf) === 100) + } + + test("FutureAction result, finite wait") { + val f = sc.parallelize(1 to 100, 4) + .countAsync() + assert(Await.result(f, Duration(30, "seconds")) === 100) + } + + test("FutureAction result, timeout") { + val f = sc.parallelize(1 to 100, 4) + .mapPartitions(itr => { Thread.sleep(20); itr }) + .countAsync() + intercept[TimeoutException] { + Await.result(f, Duration(20, "milliseconds")) + } + } } -- cgit v1.2.3 From 94b5881ee9d6c67f096ea9c2891a63978f256394 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 6 Dec 2013 00:22:00 -0800 Subject: Fix long lines --- core/src/main/scala/org/apache/spark/deploy/client/Client.scala | 6 ++++-- core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 6 ++++-- .../main/scala/org/apache/spark/storage/BlockObjectWriter.scala | 2 +- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 8 +++----- 4 files changed, 12 insertions(+), 10 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index f60e56d959..d0d65ca72a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -104,8 +104,10 @@ private[spark] class Client( activeMasterUrl = url master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) masterAddress = activeMasterUrl match { - case Master.sparkUrlRegex(host, port) => Address("akka.tcp", Master.systemName, host, port.toInt) - case x => throw new SparkException("Invalid spark URL:"+x) + case Master.sparkUrlRegex(host, port) => + Address("akka.tcp", Master.systemName, host, port.toInt) + case x => + throw new SparkException("Invalid spark URL: " + x) } } 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 808b54c0af..87531b6719 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 @@ -138,8 +138,10 @@ private[spark] class Worker( activeMasterWebUiUrl = uiUrl master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) masterAddress = activeMasterUrl match { - case Master.sparkUrlRegex(_host, _port) => Address("akka.tcp", Master.systemName, _host, _port.toInt) - case x => throw new SparkException("Invalid spark URL:"+x) + case Master.sparkUrlRegex(_host, _port) => + Address("akka.tcp", Master.systemName, _host, _port.toInt) + case x => + throw new SparkException("Invalid spark URL: " + x) } connected = true } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index df33f6bfb0..b4451fc7b8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -44,7 +44,7 @@ abstract class BlockObjectWriter(val blockId: BlockId) { * Flush the partial writes and commit them as a single atomic block. Return the * number of bytes written for this commit. */ - def commit(): LongSpark + def commit(): Long /** * Reverts writes that haven't been flushed yet. Callers should invoke this function diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 9f3f163277..74133cef6c 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -17,11 +17,8 @@ package org.apache.spark.util -import akka.actor.{IndestructibleActorSystem, ActorSystem, ExtendedActorSystem} +import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} import com.typesafe.config.ConfigFactory -import scala.concurrent.duration._ -import scala.concurrent.Await -import akka.remote.RemoteActorRefProvider /** * Various utility classes for working with Akka. @@ -47,7 +44,8 @@ private[spark] object AkkaUtils { val akkaTimeout = System.getProperty("spark.akka.timeout", "100").toInt val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt - val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" + val lifecycleEvents = + if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "600").toInt val akkaFailureDetector = -- cgit v1.2.3 From 48e4f2ad141492d7dee579a1b7fb1ec49fefa2ae Mon Sep 17 00:00:00 2001 From: "wangda.tan" Date: Mon, 9 Dec 2013 00:02:59 +0800 Subject: SPARK-968, In stage UI, add an overview section that shows task stats grouped by executor id --- .../org/apache/spark/ui/jobs/ExecutorSummary.scala | 27 +++++++ .../org/apache/spark/ui/jobs/ExecutorTable.scala | 73 ++++++++++++++++++ .../scala/org/apache/spark/ui/jobs/IndexPage.scala | 7 ++ .../apache/spark/ui/jobs/JobProgressListener.scala | 38 +++++++++ .../spark/ui/jobs/JobProgressListenerSuite.scala | 89 ++++++++++++++++++++++ 5 files changed, 234 insertions(+) create mode 100644 core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala create mode 100644 core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala create mode 100644 core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala new file mode 100644 index 0000000000..f2ee12081c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 + +private[spark] class ExecutorSummary() { + var duration : Long = 0 + var totalTasks : Int = 0 + var failedTasks : Int = 0 + var succeedTasks : Int = 0 + var shuffleRead : Long = 0 + var shuffleWrite : Long = 0 +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala new file mode 100644 index 0000000000..c6823cd823 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.xml.Node + +import org.apache.spark.scheduler.SchedulingMode + + +/** Page showing executor summary */ +private[spark] class ExecutorTable(val parent: JobProgressUI) { + + val listener = parent.listener + val dateFmt = parent.dateFmt + val isFairScheduler = listener.sc.getSchedulingMode == SchedulingMode.FAIR + + def toNodeSeq(): Seq[Node] = { + listener.synchronized { + executorTable() + } + } + + /** Special table which merges two header cells. */ + private def executorTable[T](): Seq[Node] = { + + + + + + + + + + + + {createExecutorTable()} + +
Executor IDDuration#Tasks#Failed Tasks#Succeed TasksShuffle ReadShuffle Write
+ } + + private def createExecutorTable() : Seq[Node] = { + val executorIdToSummary = listener.executorIdToSummary + executorIdToSummary.toSeq.sortBy(_._1).map{ + case (k,v) => { + + {k} + {v.duration} ms + {v.totalTasks} + {v.failedTasks} + {v.succeedTasks} + {v.shuffleRead} + {v.shuffleWrite} + + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index ca5a28625b..653a84b60f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -45,6 +45,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) val completedStagesTable = new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) + val executorTable = new ExecutorTable(parent) val pools = listener.sc.getAllPools val poolTable = new PoolTable(pools, listener) @@ -56,6 +57,10 @@ private[spark] class IndexPage(parent: JobProgressUI) { {parent.formatDuration(now - listener.sc.startTime)}
  • Scheduling Mode: {parent.sc.getSchedulingMode}
  • +
  • + Executor Summary: + {listener.executorIdToSummary.size} +
  • Active Stages: {activeStages.size} @@ -77,6 +82,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { } else { Seq() }} ++ +

    Executor Summary

    ++ + executorTable.toNodeSeq++

    Active Stages ({activeStages.size})

    ++ activeStagesTable.toNodeSeq++

    Completed Stages ({completedStages.size})

    ++ 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 6b854740d6..2635478592 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 @@ -57,6 +57,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList val stageIdToTasksFailed = HashMap[Int, Int]() val stageIdToTaskInfos = HashMap[Int, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() + val executorIdToSummary = HashMap[String, ExecutorSummary]() override def onJobStart(jobStart: SparkListenerJobStart) {} @@ -114,6 +115,9 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList sid, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) stageIdToTaskInfos(sid) = taskList + val executorSummary = executorIdToSummary.getOrElseUpdate(key = taskStart.taskInfo.executorId, + op = new ExecutorSummary()) + executorSummary.totalTasks += 1 } override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) @@ -123,9 +127,43 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + // update executor summary + val executorSummary = executorIdToSummary.get(taskEnd.taskInfo.executorId) + executorSummary match { + case Some(x) => { + // first update failed-task, succeed-task + taskEnd.reason match { + case e: ExceptionFailure => + x.failedTasks += 1 + case _ => + x.succeedTasks += 1 + } + + // update duration + x.duration += taskEnd.taskInfo.duration + + // update shuffle read/write + val shuffleRead = taskEnd.taskMetrics.shuffleReadMetrics + shuffleRead match { + case Some(s) => + x.shuffleRead += s.remoteBytesRead + case _ => {} + } + val shuffleWrite = taskEnd.taskMetrics.shuffleWriteMetrics + shuffleWrite match { + case Some(s) => { + x.shuffleWrite += s.shuffleBytesWritten + } + case _ => {} + } + } + case _ => {} + } + val sid = taskEnd.task.stageId val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) tasksActive -= taskEnd.taskInfo + val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala new file mode 100644 index 0000000000..90a58978c7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 org.scalatest.FunSuite +import org.apache.spark.scheduler._ +import org.apache.spark.SparkContext +import org.apache.spark.Success +import org.apache.spark.scheduler.SparkListenerTaskStart +import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} + +class JobProgressListenerSuite extends FunSuite { + test("test executor id to summary") { + val sc = new SparkContext("local", "joblogger") + val listener = new JobProgressListener(sc) + val taskMetrics = new TaskMetrics() + val shuffleReadMetrics = new ShuffleReadMetrics() + + // nothing in it + assert(listener.executorIdToSummary.size == 0) + + // launched a task, should get an item in map + listener.onTaskStart(new SparkListenerTaskStart( + new ShuffleMapTask(0, null, null, 0, null), + new TaskInfo(1234L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL))) + assert(listener.executorIdToSummary.size == 1) + + // finish this task, should get updated shuffleRead + shuffleReadMetrics.remoteBytesRead = 1000 + taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) + var taskInfo = new TaskInfo(1234L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL) + taskInfo.finishTime = 1 + listener.onTaskEnd(new SparkListenerTaskEnd( + new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + assert(listener.executorIdToSummary.getOrElse("exe-1", fail()).shuffleRead == 1000) + + // finish a task with unknown executor-id, nothing should happen + taskInfo = new TaskInfo(1234L, 0, 1000L, "exe-unknown", "host1", TaskLocality.NODE_LOCAL) + taskInfo.finishTime = 1 + listener.onTaskEnd(new SparkListenerTaskEnd( + new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + assert(listener.executorIdToSummary.size == 1) + + // launched a task + listener.onTaskStart(new SparkListenerTaskStart( + new ShuffleMapTask(0, null, null, 0, null), + new TaskInfo(1235L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL))) + assert(listener.executorIdToSummary.size == 1) + + // finish this task, should get updated duration + shuffleReadMetrics.remoteBytesRead = 1000 + taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) + taskInfo = new TaskInfo(1235L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL) + taskInfo.finishTime = 1 + listener.onTaskEnd(new SparkListenerTaskEnd( + new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + assert(listener.executorIdToSummary.getOrElse("exe-1", fail()).shuffleRead == 2000) + + // launched a task in another exec + listener.onTaskStart(new SparkListenerTaskStart( + new ShuffleMapTask(0, null, null, 0, null), + new TaskInfo(1236L, 0, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL))) + assert(listener.executorIdToSummary.size == 2) + + // finish this task, should get updated duration + shuffleReadMetrics.remoteBytesRead = 1000 + taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) + taskInfo = new TaskInfo(1236L, 0, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL) + taskInfo.finishTime = 1 + listener.onTaskEnd(new SparkListenerTaskEnd( + new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + assert(listener.executorIdToSummary.getOrElse("exe-2", fail()).shuffleRead == 1000) + } +} -- cgit v1.2.3 From ee68a85cff499c7aa5d448cc72a93e4de3c23c41 Mon Sep 17 00:00:00 2001 From: "wangda.tan" Date: Mon, 9 Dec 2013 09:38:58 +0800 Subject: SPARK-968, added sc finalize code to avoid akka rebinding to the same port --- .../scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala | 7 +++++++ 1 file changed, 7 insertions(+) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 90a58978c7..861d37a862 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -85,5 +85,12 @@ class JobProgressListenerSuite extends FunSuite { listener.onTaskEnd(new SparkListenerTaskEnd( new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) assert(listener.executorIdToSummary.getOrElse("exe-2", fail()).shuffleRead == 1000) + + // do finalize + sc.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") } } -- cgit v1.2.3 From 14a1df65729a3c7e5fd54f79424626a540aef5d4 Mon Sep 17 00:00:00 2001 From: Grega Kespret Date: Mon, 9 Dec 2013 10:39:02 +0100 Subject: Fix for spark.task.maxFailures not enforced correctly. --- .../org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) (limited to 'core/src') 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 94961790df..bf494aa64d 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 @@ -529,10 +529,10 @@ private[spark] class ClusterTaskSetManager( addPendingTask(index) if (state != TaskState.KILLED) { numFailures(index) += 1 - if (numFailures(index) > MAX_TASK_FAILURES) { - logError("Task %s:%d failed more than %d times; aborting job".format( + if (numFailures(index) >= MAX_TASK_FAILURES) { + logError("Task %s:%d failed %d times; aborting job".format( taskSet.id, index, MAX_TASK_FAILURES)) - abort("Task %s:%d failed more than %d times".format(taskSet.id, index, MAX_TASK_FAILURES)) + abort("Task %s:%d failed %d times".format(taskSet.id, index, MAX_TASK_FAILURES)) } } } else { -- cgit v1.2.3 From 5b74609d97cce9c350e959547e7fc524182a457c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 9 Dec 2013 16:40:30 -0800 Subject: License headers --- .../org/apache/spark/storage/StoragePerfTester.scala | 17 +++++++++++++++++ .../apache/spark/deploy/worker/ExecutorRunnerTest.scala | 17 +++++++++++++++++ .../apache/spark/storage/DiskBlockManagerSuite.scala | 17 +++++++++++++++++ .../apache/spark/util/collection/OpenHashMapSuite.scala | 17 +++++++++++++++++ .../apache/spark/util/collection/OpenHashSetSuite.scala | 17 +++++++++++++++++ .../util/collection/PrimitiveKeyOpenHashMapSuite.scala | 17 +++++++++++++++++ python/test_support/userlibrary.py | 17 +++++++++++++++++ 7 files changed, 119 insertions(+) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala index 1e4db4f66b..d52b3d8284 100644 --- a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala +++ b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 java.util.concurrent.atomic.AtomicLong diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 8f0954122b..4cb4ddc9cd 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 java.io.File diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index ef4c4c0f14..070982e798 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 java.io.{FileWriter, File} diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala index 63e874fed3..e9b62ea70d 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.util.collection import scala.collection.mutable.HashSet diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala index 4768a1e60b..1b24f8f287 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.util.collection import org.scalatest.FunSuite diff --git a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala index 2220b4f0d5..3b60decee9 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.util.collection import scala.collection.mutable.HashSet diff --git a/python/test_support/userlibrary.py b/python/test_support/userlibrary.py index 5bb6f5009f..8e4a6292bc 100755 --- a/python/test_support/userlibrary.py +++ b/python/test_support/userlibrary.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. +# + """ Used to test shipping of code depenencies with SparkContext.addPyFile(). """ -- cgit v1.2.3 From 17db6a9041d5e83d7b6fe47f9c36758d0613fcd6 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 10 Dec 2013 11:34:10 +0530 Subject: Style fixes and addressed review comments at #221 --- assembly/pom.xml | 14 +++++++------- bagel/pom.xml | 10 +++++----- core/pom.xml | 18 +++++++++--------- .../scala/org/apache/spark/MapOutputTracker.scala | 9 ++++----- .../main/scala/org/apache/spark/Partitioner.scala | 6 +++--- .../apache/spark/deploy/worker/ui/WorkerWebUI.scala | 14 +++++--------- .../scala/org/apache/spark/rdd/CheckpointRDD.scala | 11 ++++------- .../main/scala/org/apache/spark/rdd/JdbcRDD.scala | 1 + .../main/scala/org/apache/spark/rdd/MappedRDD.scala | 3 ++- .../org/apache/spark/rdd/OrderedRDDFunctions.scala | 3 ++- .../scala/org/apache/spark/rdd/ShuffledRDD.scala | 3 ++- .../apache/spark/storage/BlockManagerMaster.scala | 7 +++---- .../spark/util/IndestructibleActorSystem.scala | 17 +++++++++++++++-- examples/pom.xml | 20 ++++++++++---------- mllib/pom.xml | 10 +++++----- pom.xml | 17 ++++++++--------- repl-bin/pom.xml | 6 +++--- repl/pom.xml | 14 +++++++------- streaming/pom.xml | 16 ++++++++-------- .../spark/streaming/receivers/ZeroMQReceiver.scala | 4 ++-- tools/pom.xml | 10 +++++----- yarn/pom.xml | 8 ++++---- 22 files changed, 114 insertions(+), 107 deletions(-) (limited to 'core/src') diff --git a/assembly/pom.xml b/assembly/pom.xml index 8103534796..c2cda41c6d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -41,27 +41,27 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} org.apache.spark - spark-bagel_${scala-short.version} + spark-bagel_2.10 ${project.version} org.apache.spark - spark-mllib_${scala-short.version} + spark-mllib_2.10 ${project.version} org.apache.spark - spark-repl_${scala-short.version} + spark-repl_2.10 ${project.version} org.apache.spark - spark-streaming_${scala-short.version} + spark-streaming_2.10 ${project.version} @@ -79,7 +79,7 @@ maven-shade-plugin false - ${project.build.directory}/scala-${scala-short.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar + ${project.build.directory}/scala-2.10/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar *:* @@ -128,7 +128,7 @@ org.apache.spark - spark-yarn_${scala-short.version} + spark-yarn_2.10 ${project.version} diff --git a/bagel/pom.xml b/bagel/pom.xml index 461e76a753..0f550d70d6 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -34,7 +34,7 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} @@ -43,18 +43,18 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.scalatest diff --git a/core/pom.xml b/core/pom.xml index af605e1837..71bf15c491 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -86,7 +86,7 @@ com.twitter - chill_${scala-short.version} + chill_2.10 0.3.1 @@ -96,11 +96,11 @@ com.typesafe.akka - akka-remote_${scala-short.version} + akka-remote_2.10 com.typesafe.akka - akka-slf4j_${scala-short.version} + akka-slf4j_2.10 org.scala-lang @@ -108,7 +108,7 @@ net.liftweb - lift-json_${scala-short.version} + lift-json_2.10 it.unimi.dsi @@ -120,7 +120,7 @@ com.github.scala-incubator.io - scala-io-file_${scala-short.version} + scala-io-file_2.10 org.apache.mesos @@ -166,12 +166,12 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test @@ -191,8 +191,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 88a7f24884..d36e1b13a6 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -72,12 +72,11 @@ private[spark] class MapOutputTracker extends Logging { // throw a SparkException if this fails. private def askTracker(message: Any): Any = { try { - val future = if (trackerActor.isLeft ) { - trackerActor.left.get.ask(message)(timeout) - } else { - trackerActor.right.get.ask(message)(timeout) + val future = trackerActor match { + case Left(a: ActorRef) => a.ask(message)(timeout) + case Right(b: ActorSelection) => b.ask(message)(timeout) } - return Await.result(future, timeout) + Await.result(future, timeout) } catch { case e: Exception => throw new SparkException("Error communicating with MapOutputTracker", e) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 62b608c088..bcec41c439 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -17,11 +17,11 @@ package org.apache.spark -import org.apache.spark.util.Utils -import org.apache.spark.rdd.RDD - import scala.reflect.ClassTag +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + /** * 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/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index a38e32b339..6c18a3c245 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 @@ -17,23 +17,19 @@ package org.apache.spark.deploy.worker.ui -import akka.actor.ActorRef -import akka.util.Timeout +import java.io.File import scala.concurrent.duration._ -import java.io.{FileInputStream, File} - +import akka.util.Timeout import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.{Handler, Server} - +import org.apache.spark.Logging import org.apache.spark.deploy.worker.Worker -import org.apache.spark.{Logging} -import org.apache.spark.ui.JettyUtils +import org.apache.spark.ui.{JettyUtils, UIUtils} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils +import org.eclipse.jetty.server.{Handler, Server} /** * Web UI server for the standalone worker. diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 99ea6e8ee8..a712ef1c27 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -17,16 +17,13 @@ package org.apache.spark.rdd +import java.io.IOException + import scala.reflect.ClassTag + +import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.hadoop.mapred.{FileInputFormat, SequenceFileInputFormat, JobConf, Reporter} -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.io.{NullWritable, BytesWritable} -import org.apache.hadoop.util.ReflectionUtils -import org.apache.hadoop.fs.Path -import java.io.{File, IOException, EOFException} -import java.text.NumberFormat private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {} 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 e72f86fb13..8df8718f3b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import java.sql.{Connection, ResultSet} import scala.reflect.ClassTag + import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} import org.apache.spark.util.NextIterator 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 eb3b19907d..8d7c288593 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala @@ -17,9 +17,10 @@ package org.apache.spark.rdd -import org.apache.spark.{Partition, TaskContext} import scala.reflect.ClassTag +import org.apache.spark.{Partition, TaskContext} + private[spark] class MappedRDD[U: ClassTag, T: ClassTag](prev: RDD[T], f: T => U) extends RDD[U](prev) { 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 4a465840c6..d5691f2267 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -17,9 +17,10 @@ package org.apache.spark.rdd -import org.apache.spark.{RangePartitioner, Logging} import scala.reflect.ClassTag +import org.apache.spark.{Logging, RangePartitioner} + /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through * an implicit conversion. Import `org.apache.spark.SparkContext._` at the top of your program to 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 1d109a2496..3682c84598 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -17,9 +17,10 @@ package org.apache.spark.rdd -import org.apache.spark.{Dependency, Partitioner, SparkEnv, ShuffleDependency, Partition, TaskContext} import scala.reflect.ClassTag +import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency, + SparkEnv, TaskContext} private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { override val index = idx diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index e5de16fc01..e05b842476 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -157,10 +157,9 @@ private[spark] class BlockManagerMaster(var driverActor : Either[ActorRef, Actor while (attempts < AKKA_RETRY_ATTEMPTS) { attempts += 1 try { - val future = if (driverActor.isLeft ) { - driverActor.left.get.ask(message)(timeout) - } else { - driverActor.right.get.ask(message)(timeout) + val future = driverActor match { + case Left(a: ActorRef) => a.ask(message)(timeout) + case Right(b: ActorSelection) => b.ask(message)(timeout) } val result = Await.result(future, timeout) if (result == null) { diff --git a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala index 69519860c6..bf71882ef7 100644 --- a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala +++ b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala @@ -1,5 +1,18 @@ -/** - * Copyright (C) 2009-2013 Typesafe Inc. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ // Must be in akka.actor package as ActorSystemImpl is protected[akka]. diff --git a/examples/pom.xml b/examples/pom.xml index 464ad82e33..97f6dfea66 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -49,25 +49,25 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} provided org.apache.spark - spark-streaming_${scala-short.version} + spark-streaming_2.10 ${project.version} provided org.apache.spark - spark-mllib_${scala-short.version} + spark-mllib_2.10 ${project.version} provided org.apache.spark - spark-bagel_${scala-short.version} + spark-bagel_2.10 ${project.version} provided @@ -88,7 +88,7 @@ org.apache.kafka - kafka_2.9.2 + kafka_2.10 0.8.0-beta1 @@ -107,17 +107,17 @@ com.twitter - algebird-core_${scala-short.version} + algebird-core_2.10 0.1.11 org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test @@ -166,8 +166,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins diff --git a/mllib/pom.xml b/mllib/pom.xml index fce5b19bb2..228f8c029b 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -34,7 +34,7 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} @@ -48,12 +48,12 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test @@ -63,8 +63,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.scalatest diff --git a/pom.xml b/pom.xml index 4c11459088..979fd0c287 100644 --- a/pom.xml +++ b/pom.xml @@ -101,7 +101,6 @@ 1.6 - 2.10 2.10.3 0.13.0 2.2.3 @@ -205,7 +204,7 @@ com.twitter - chill_${scala-short.version} + chill_2.10 0.3.1 @@ -215,7 +214,7 @@ com.typesafe.akka - akka-actor_${scala-short.version} + akka-actor_2.10 ${akka.version} @@ -226,7 +225,7 @@ com.typesafe.akka - akka-remote_${scala-short.version} + akka-remote_2.10 ${akka.version} @@ -237,7 +236,7 @@ com.typesafe.akka - akka-slf4j_${scala-short.version} + akka-slf4j_2.10 ${akka.version} @@ -258,7 +257,7 @@ com.github.scala-incubator.io - scala-io-file_${scala-short.version} + scala-io-file_2.10 0.4.1 @@ -279,7 +278,7 @@ net.liftweb - lift-json_${scala-short.version} + lift-json_2.10 2.5.1 @@ -335,7 +334,7 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 1.9.1 test @@ -358,7 +357,7 @@ org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 1.10.0 test diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index f8a17d93cc..c2a4efa36c 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -40,18 +40,18 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} org.apache.spark - spark-bagel_${scala-short.version} + spark-bagel_2.10 ${project.version} runtime org.apache.spark - spark-repl_${scala-short.version} + spark-repl_2.10 ${project.version} runtime diff --git a/repl/pom.xml b/repl/pom.xml index 2f27e76760..bf06d730d4 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -39,18 +39,18 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} org.apache.spark - spark-bagel_${scala-short.version} + spark-bagel_2.10 ${project.version} runtime org.apache.spark - spark-mllib_${scala-short.version} + spark-mllib_2.10 ${project.version} runtime @@ -78,18 +78,18 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins diff --git a/streaming/pom.xml b/streaming/pom.xml index ff95591b9f..298bc83161 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -48,7 +48,7 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} @@ -61,8 +61,8 @@ 1.9.11 - org.apache.kafka - kafka_2.9.2 + com.sksamuel.kafka + kafka_2.10 0.8.0-beta1 @@ -111,7 +111,7 @@ com.typesafe.akka - akka-zeromq_${scala-short.version} + akka-zeromq_2.10 ${akka.version} @@ -122,12 +122,12 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test @@ -151,8 +151,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.scalatest diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala index ce8c56fa8a..f164d516b0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala @@ -17,14 +17,14 @@ package org.apache.spark.streaming.receivers +import scala.reflect.ClassTag + import akka.actor.Actor import akka.util.ByteString import akka.zeromq._ import org.apache.spark.Logging -import scala.reflect.ClassTag - /** * A receiver to subscribe to ZeroMQ stream. */ diff --git a/tools/pom.xml b/tools/pom.xml index 353d201f90..2bad494fc5 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -33,24 +33,24 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} org.apache.spark - spark-streaming_${scala-short.version} + spark-streaming_2.10 ${project.version} org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins diff --git a/yarn/pom.xml b/yarn/pom.xml index 5cf81575c9..d18ac3736d 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -33,7 +33,7 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} @@ -63,7 +63,7 @@ org.scalatest - scalatest_2.9.3 + scalatest_2.10 test @@ -74,8 +74,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins -- cgit v1.2.3 From 5013fb64b27e46bbc5daf4f06fdc70938c06cf29 Mon Sep 17 00:00:00 2001 From: Binh Nguyen Date: Tue, 10 Dec 2013 00:38:16 -0800 Subject: Expose numPartitions parameter in JavaPairRDD.sortByKey() This change make Java and Scala API on sortByKey() the same. --- .../main/scala/org/apache/spark/api/java/JavaPairRDD.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) (limited to 'core/src') 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 2142fd7327..a191dfd1dc 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 @@ -583,12 +583,20 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in * order of the keys). */ - def sortByKey(comp: Comparator[K], ascending: Boolean): JavaPairRDD[K, V] = { + def sortByKey(comp: Comparator[K], ascending: Boolean): JavaPairRDD[K, V] = sortByKey(comp, ascending, rdd.partitions.size) + + /** + * 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(comp: Comparator[K], ascending: Boolean, numPartitions: Int): JavaPairRDD[K, V] = { class KeyOrdering(val a: K) extends Ordered[K] { override def compare(b: K) = comp.compare(a, b) } implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x) - fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending)) + fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending, numPartitions)) } /** -- cgit v1.2.3 From c82d4f079bf84da06801419defe644f574d0b8d1 Mon Sep 17 00:00:00 2001 From: Binh Nguyen Date: Tue, 10 Dec 2013 01:04:52 -0800 Subject: Use braces to shorten the line. --- core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) (limited to 'core/src') 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 a191dfd1dc..36bd3e673f 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 @@ -583,7 +583,9 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in * order of the keys). */ - def sortByKey(comp: Comparator[K], ascending: Boolean): JavaPairRDD[K, V] = sortByKey(comp, ascending, rdd.partitions.size) + def sortByKey(comp: Comparator[K], ascending: Boolean): JavaPairRDD[K, V] = { + sortByKey(comp, ascending, rdd.partitions.size) + } /** * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling -- cgit v1.2.3 From 558af873340087cad79630ec5c498672c5ea3c4f Mon Sep 17 00:00:00 2001 From: Grega Kespret Date: Tue, 10 Dec 2013 11:43:42 +0100 Subject: Fix tests. --- core/src/test/scala/org/apache/spark/DistributedSuite.scala | 2 +- .../org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 480bac84f3..816cbee3e0 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -122,7 +122,7 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter sc.parallelize(1 to 10, 10).foreach(x => println(x / 0)) } assert(thrown.getClass === classOf[SparkException]) - assert(thrown.getMessage.contains("more than 4 times")) + assert(thrown.getMessage.contains("failed 4 times")) } test("caching") { 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 b97f2b19b5..29c4cc5d9c 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 @@ -283,7 +283,7 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted // after the last failure. - (0 until manager.MAX_TASK_FAILURES).foreach { index => + (1 to manager.MAX_TASK_FAILURES).foreach { index => val offerResult = manager.resourceOffer("exec1", "host1", 1, ANY) assert(offerResult != None, "Expect resource offer on iteration %s to return a task".format(index)) -- cgit v1.2.3 From e85af507671d417724c28ee2db499fc019feb1d8 Mon Sep 17 00:00:00 2001 From: Binh Nguyen Date: Tue, 10 Dec 2013 11:01:56 -0800 Subject: Leave default value of numPartitions to Scala code. --- .../src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) (limited to 'core/src') 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 36bd3e673f..2d2b3847de 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 @@ -584,7 +584,9 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * order of the keys). */ def sortByKey(comp: Comparator[K], ascending: Boolean): JavaPairRDD[K, V] = { - sortByKey(comp, ascending, rdd.partitions.size) + // numPartitions should never be negative in practice so we can use -1 here to indicate that + // we want to use implementation's default value. + sortByKey(comp, ascending, -1) } /** @@ -598,7 +600,11 @@ 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[K, V, (K, V)](rdd).sortByKey(ascending, numPartitions)) + if (numPartitions < 0) { + fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending)) + } else { + fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending, numPartitions)) + } } /** -- cgit v1.2.3 From 0b494f7db47cf1de35aaed046f21bbb3592c3d97 Mon Sep 17 00:00:00 2001 From: Binh Nguyen Date: Tue, 10 Dec 2013 11:17:52 -0800 Subject: Hook directly to Scala API --- .../main/scala/org/apache/spark/api/java/JavaPairRDD.scala | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) (limited to 'core/src') 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 2d2b3847de..d0ca289a6c 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 @@ -584,9 +584,11 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * order of the keys). */ def sortByKey(comp: Comparator[K], ascending: Boolean): JavaPairRDD[K, V] = { - // numPartitions should never be negative in practice so we can use -1 here to indicate that - // we want to use implementation's default value. - sortByKey(comp, ascending, -1) + class KeyOrdering(val a: K) extends Ordered[K] { + override def compare(b: K) = comp.compare(a, b) + } + implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x) + fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending)) } /** @@ -600,11 +602,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) - if (numPartitions < 0) { - fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending)) - } else { - fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending, numPartitions)) - } + fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending, numPartitions)) } /** -- cgit v1.2.3 From 49bf47e1b792b82561b164f4f8006ddd4dd350ee Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Tue, 10 Dec 2013 19:50:50 -0800 Subject: Removed superfluous abs call from test cases. --- core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 5683ada78a..6ad58b875d 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -126,7 +126,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val rdd1 = sc.parallelize(stacked) val counted1 = rdd1.countDistinctByKey(relativeSD).collect() counted1.foreach{ - case(k, count) => assert(math.abs(error(count, k)) < relativeSD) + case(k, count) => assert(error(count, k) < relativeSD) } val rnd = new Random() @@ -139,7 +139,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val rdd2 = sc.parallelize(randStacked) val counted2 = rdd2.countDistinctByKey(relativeSD, 4).collect() counted2.foreach{ - case(k, count) => assert(math.abs(error(count, k)) < relativeSD) + case(k, count) => assert(error(count, k) < relativeSD) } } -- cgit v1.2.3 From 5e9ce83d682d6198cda4631faf11cb53fcccf07f Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 11 Dec 2013 14:01:36 -0800 Subject: Fixed multiple file stream and checkpointing bugs. - Made file stream more robust to transient failures. - Changed Spark.setCheckpointDir API to not have the second 'useExisting' parameter. Spark will always create a unique directory for checkpointing underneath the directory provide to the funtion. - Fixed bug wrt local relative paths as checkpoint directory. - Made DStream and RDD checkpointing use SparkContext.hadoopConfiguration, so that more HDFS compatible filesystems are supported for checkpointing. --- .../main/scala/org/apache/spark/SparkContext.scala | 25 ++-- .../apache/spark/api/java/JavaSparkContext.scala | 15 +-- .../scala/org/apache/spark/rdd/CheckpointRDD.scala | 27 +++-- .../org/apache/spark/rdd/RDDCheckpointData.scala | 14 ++- .../test/scala/org/apache/spark/JavaAPISuite.java | 4 +- .../org/apache/spark/streaming/Checkpoint.scala | 6 +- .../org/apache/spark/streaming/Scheduler.scala | 2 +- .../apache/spark/streaming/StreamingContext.scala | 15 ++- .../spark/streaming/dstream/FileInputDStream.scala | 130 +++++++++++++-------- .../apache/spark/streaming/CheckpointSuite.scala | 38 +++--- 10 files changed, 159 insertions(+), 117 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 66006bf212..1811bfa1e5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -19,7 +19,7 @@ package org.apache.spark import java.io._ import java.net.URI -import java.util.Properties +import java.util.{UUID, Properties} import java.util.concurrent.atomic.AtomicInteger import scala.collection.Map @@ -857,22 +857,15 @@ class SparkContext( /** * Set the directory under which RDDs are going to be checkpointed. The directory must - * be a HDFS path if running on a cluster. If the directory does not exist, it will - * be created. If the directory exists and useExisting is set to true, then the - * exisiting directory will be used. Otherwise an exception will be thrown to - * prevent accidental overriding of checkpoint files in the existing directory. + * be a HDFS path if running on a cluster. */ - def setCheckpointDir(dir: String, useExisting: Boolean = false) { - val path = new Path(dir) - val fs = path.getFileSystem(SparkHadoopUtil.get.newConfiguration()) - if (!useExisting) { - if (fs.exists(path)) { - throw new Exception("Checkpoint directory '" + path + "' already exists.") - } else { - fs.mkdirs(path) - } - } - checkpointDir = Some(dir) + def setCheckpointDir(directory: String) { + checkpointDir = Option(directory).map(dir => { + val path = new Path(dir, UUID.randomUUID().toString) + val fs = path.getFileSystem(hadoopConfiguration) + fs.mkdirs(path) + fs.getFileStatus(path).getPath().toString + }) } /** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */ 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 8869e072bf..c63db4970b 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 @@ -385,20 +385,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork /** * Set the directory under which RDDs are going to be checkpointed. The directory must - * be a HDFS path if running on a cluster. If the directory does not exist, it will - * be created. If the directory exists and useExisting is set to true, then the - * exisiting directory will be used. Otherwise an exception will be thrown to - * prevent accidental overriding of checkpoint files in the existing directory. - */ - def setCheckpointDir(dir: String, useExisting: Boolean) { - sc.setCheckpointDir(dir, useExisting) - } - - /** - * Set the directory under which RDDs are going to be checkpointed. The directory must - * be a HDFS path if running on a cluster. If the directory does not exist, it will - * be created. If the directory exists, an exception will be thrown to prevent accidental - * overriding of checkpoint files. + * be a HDFS path if running on a cluster. */ def setCheckpointDir(dir: String) { sc.setCheckpointDir(dir) diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index d3033ea4a6..ef4057e2a2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.util.ReflectionUtils import org.apache.hadoop.fs.Path import java.io.{File, IOException, EOFException} import java.text.NumberFormat +import org.apache.spark.broadcast.Broadcast private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {} @@ -36,6 +37,8 @@ private[spark] class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: String) extends RDD[T](sc, Nil) { + val broadcastedConf = sc.broadcast(new SerializableWritable(sc.hadoopConfiguration)) + @transient val fs = new Path(checkpointPath).getFileSystem(sc.hadoopConfiguration) override def getPartitions: Array[Partition] = { @@ -67,7 +70,7 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri override def compute(split: Partition, context: TaskContext): Iterator[T] = { val file = new Path(checkpointPath, CheckpointRDD.splitIdToFile(split.index)) - CheckpointRDD.readFromFile(file, context) + CheckpointRDD.readFromFile(file, broadcastedConf, context) } override def checkpoint() { @@ -81,10 +84,14 @@ private[spark] object CheckpointRDD extends Logging { "part-%05d".format(splitId) } - def writeToFile[T](path: String, blockSize: Int = -1)(ctx: TaskContext, iterator: Iterator[T]) { + def writeToFile[T]( + path: String, + broadcastedConf: Broadcast[SerializableWritable[Configuration]], + blockSize: Int = -1 + )(ctx: TaskContext, iterator: Iterator[T]) { val env = SparkEnv.get val outputDir = new Path(path) - val fs = outputDir.getFileSystem(SparkHadoopUtil.get.newConfiguration()) + val fs = outputDir.getFileSystem(broadcastedConf.value.value) val finalOutputName = splitIdToFile(ctx.partitionId) val finalOutputPath = new Path(outputDir, finalOutputName) @@ -121,9 +128,13 @@ private[spark] object CheckpointRDD extends Logging { } } - def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = { + def readFromFile[T]( + path: Path, + broadcastedConf: Broadcast[SerializableWritable[Configuration]], + context: TaskContext + ): Iterator[T] = { val env = SparkEnv.get - val fs = path.getFileSystem(SparkHadoopUtil.get.newConfiguration()) + val fs = path.getFileSystem(broadcastedConf.value.value) val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt val fileInputStream = fs.open(path, bufferSize) val serializer = env.serializer.newInstance() @@ -146,8 +157,10 @@ private[spark] object CheckpointRDD extends Logging { 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.get.newConfiguration()) - sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, 1024) _) + val conf = SparkHadoopUtil.get.newConfiguration() + val fs = path.getFileSystem(conf) + val broadcastedConf = sc.broadcast(new SerializableWritable(conf)) + sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, broadcastedConf, 1024) _) val cpRDD = new CheckpointRDD[Int](sc, path.toString) assert(cpRDD.partitions.length == rdd.partitions.length, "Number of partitions is not the same") assert(cpRDD.collect.toList == rdd.collect.toList, "Data of partitions not the same") diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index 6009a41570..3160ab95c4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -20,7 +20,7 @@ package org.apache.spark.rdd import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration -import org.apache.spark.{Partition, SparkException, Logging} +import org.apache.spark.{SerializableWritable, Partition, SparkException, Logging} import org.apache.spark.scheduler.{ResultTask, ShuffleMapTask} /** @@ -83,14 +83,20 @@ private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T]) // Create the output path for the checkpoint val path = new Path(rdd.context.checkpointDir.get, "rdd-" + rdd.id) - val fs = path.getFileSystem(new Configuration()) + val fs = path.getFileSystem(rdd.context.hadoopConfiguration) if (!fs.mkdirs(path)) { throw new SparkException("Failed to create checkpoint path " + path) } // Save to file, and reload it as an RDD - rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path.toString) _) + val broadcastedConf = rdd.context.broadcast(new SerializableWritable(rdd.context.hadoopConfiguration)) + rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path.toString, broadcastedConf) _) val newRDD = new CheckpointRDD[T](rdd.context, path.toString) + if (newRDD.partitions.size != rdd.partitions.size) { + throw new Exception( + "Checkpoint RDD " + newRDD + "("+ newRDD.partitions.size + ") has different " + + "number of partitions than original RDD " + rdd + "(" + rdd.partitions.size + ")") + } // Change the dependencies and partitions of the RDD RDDCheckpointData.synchronized { @@ -99,8 +105,8 @@ private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T]) rdd.markCheckpointed(newRDD) // Update the RDD's dependencies and partitions cpState = Checkpointed RDDCheckpointData.clearTaskCaches() - logInfo("Done checkpointing RDD " + rdd.id + ", new parent is RDD " + newRDD.id) } + logInfo("Done checkpointing RDD " + rdd.id + " to " + path + ", new parent is RDD " + newRDD.id) } // Get preferred location of a split after checkpointing diff --git a/core/src/test/scala/org/apache/spark/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java index 4234f6eac7..ee5d8c9f13 100644 --- a/core/src/test/scala/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java @@ -851,7 +851,7 @@ public class JavaAPISuite implements Serializable { public void checkpointAndComputation() { File tempDir = Files.createTempDir(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - sc.setCheckpointDir(tempDir.getAbsolutePath(), true); + sc.setCheckpointDir(tempDir.getAbsolutePath()); Assert.assertEquals(false, rdd.isCheckpointed()); rdd.checkpoint(); rdd.count(); // Forces the DAG to cause a checkpoint @@ -863,7 +863,7 @@ public class JavaAPISuite implements Serializable { public void checkpointAndRestore() { File tempDir = Files.createTempDir(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - sc.setCheckpointDir(tempDir.getAbsolutePath(), true); + sc.setCheckpointDir(tempDir.getAbsolutePath()); Assert.assertEquals(false, rdd.isCheckpointed()); rdd.checkpoint(); rdd.count(); // Forces the DAG to cause a checkpoint diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 9271914eb5..bcf5e6b1e6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.Logging import org.apache.spark.io.CompressionCodec import org.apache.spark.util.MetadataCleaner +import org.apache.spark.deploy.SparkHadoopUtil private[streaming] @@ -57,7 +58,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) * Convenience class to speed up the writing of graph checkpoint to file */ private[streaming] -class CheckpointWriter(checkpointDir: String) extends Logging { +class CheckpointWriter(checkpointDir: String, hadoopConf: Configuration) extends Logging { val file = new Path(checkpointDir, "graph") // The file to which we actually write - and then "move" to file. private val writeFile = new Path(file.getParent, file.getName + ".next") @@ -65,8 +66,7 @@ class CheckpointWriter(checkpointDir: String) extends Logging { private var stopped = false - val conf = new Configuration() - var fs = file.getFileSystem(conf) + var fs = file.getFileSystem(hadoopConf) val maxAttempts = 3 val executor = Executors.newFixedThreadPool(1) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala index ed892e33e6..4cd8695df5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala @@ -29,7 +29,7 @@ class Scheduler(ssc: StreamingContext) extends Logging { val concurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt val jobManager = new JobManager(ssc, concurrentJobs) val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) { - new CheckpointWriter(ssc.checkpointDir) + new CheckpointWriter(ssc.checkpointDir, ssc.sparkContext.hadoopConfiguration) } else { null } 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 70bf902143..d6fc2a19f4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -46,6 +46,7 @@ import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path import twitter4j.Status import twitter4j.auth.Authorization +import org.apache.spark.deploy.SparkHadoopUtil /** @@ -85,7 +86,6 @@ class StreamingContext private ( null, batchDuration) } - /** * Re-create a StreamingContext from a checkpoint file. * @param path Path either to the directory that was specified as the checkpoint directory, or @@ -139,7 +139,7 @@ class StreamingContext private ( protected[streaming] var checkpointDir: String = { if (isCheckpointPresent) { - sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(cp_.checkpointDir), true) + sc.setCheckpointDir(cp_.checkpointDir) cp_.checkpointDir } else { null @@ -173,8 +173,12 @@ class StreamingContext private ( */ def checkpoint(directory: String) { if (directory != null) { - sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(directory)) - checkpointDir = directory + val path = new Path(directory) + val fs = path.getFileSystem(sparkContext.hadoopConfiguration) + fs.mkdirs(path) + val fullPath = fs.getFileStatus(path).getPath().toString + sc.setCheckpointDir(fullPath) + checkpointDir = fullPath } else { checkpointDir = null } @@ -595,8 +599,9 @@ object StreamingContext { prefix + "-" + time.milliseconds + "." + suffix } } - + /* protected[streaming] def getSparkCheckpointDir(sscCheckpointDir: String): String = { new Path(sscCheckpointDir, UUID.randomUUID.toString).toString } + */ } 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 fea0573b77..1a8db3ab59 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 @@ -39,8 +39,8 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData // Latest file mod time seen till any point of time - private val lastModTimeFiles = new HashSet[String]() - private var lastModTime = 0L + private val prevModTimeFiles = new HashSet[String]() + private var prevModTime = 0L @transient private var path_ : Path = null @transient private var fs_ : FileSystem = null @@ -48,11 +48,11 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K override def start() { if (newFilesOnly) { - lastModTime = graph.zeroTime.milliseconds + prevModTime = graph.zeroTime.milliseconds } else { - lastModTime = 0 + prevModTime = 0 } - logDebug("LastModTime initialized to " + lastModTime + ", new files only = " + newFilesOnly) + logDebug("LastModTime initialized to " + prevModTime + ", new files only = " + newFilesOnly) } override def stop() { } @@ -67,55 +67,22 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K * the previous call. */ override def compute(validTime: Time): Option[RDD[(K, V)]] = { - assert(validTime.milliseconds >= lastModTime, "Trying to get new files for really old time [" + validTime + " < " + lastModTime) - - // Create the filter for selecting new files - val newFilter = new PathFilter() { - // Latest file mod time seen in this round of fetching files and its corresponding files - var latestModTime = 0L - val latestModTimeFiles = new HashSet[String]() - - def accept(path: Path): Boolean = { - if (!filter(path)) { // Reject file if it does not satisfy filter - logDebug("Rejected by filter " + path) - return false - } else { // Accept file only if - val modTime = fs.getFileStatus(path).getModificationTime() - logDebug("Mod time for " + path + " is " + modTime) - if (modTime < lastModTime) { - logDebug("Mod time less than last mod time") - return false // If the file was created before the last time it was called - } else if (modTime == lastModTime && lastModTimeFiles.contains(path.toString)) { - logDebug("Mod time equal to last mod time, but file considered already") - return false // If the file was created exactly as lastModTime but not reported yet - } else if (modTime > validTime.milliseconds) { - logDebug("Mod time more than valid time") - return false // If the file was created after the time this function call requires - } - if (modTime > latestModTime) { - latestModTime = modTime - latestModTimeFiles.clear() - logDebug("Latest mod time updated to " + latestModTime) - } - latestModTimeFiles += path.toString - logDebug("Accepted " + path) - return true - } - } - } - logDebug("Finding new files at time " + validTime + " for last mod time = " + lastModTime) - val newFiles = fs.listStatus(path, newFilter).map(_.getPath.toString) + assert(validTime.milliseconds >= prevModTime, + "Trying to get new files for really old time [" + validTime + " < " + prevModTime) + + // Find new files + val (newFiles, latestModTime, latestModTimeFiles) = findNewFiles(validTime.milliseconds) logInfo("New files at time " + validTime + ":\n" + newFiles.mkString("\n")) if (newFiles.length > 0) { // Update the modification time and the files processed for that modification time - if (lastModTime != newFilter.latestModTime) { - lastModTime = newFilter.latestModTime - lastModTimeFiles.clear() + if (prevModTime < latestModTime) { + prevModTime = latestModTime + prevModTimeFiles.clear() } - lastModTimeFiles ++= newFilter.latestModTimeFiles - logDebug("Last mod time updated to " + lastModTime) + prevModTimeFiles ++= latestModTimeFiles + logDebug("Last mod time updated to " + prevModTime) } - files += ((validTime, newFiles)) + files += ((validTime, newFiles.toArray)) Some(filesToRDD(newFiles)) } @@ -130,8 +97,30 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K oldFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n")) } + /** + * Finds files which have modification timestamp <= current time. If some files are being + * deleted in the directory, then it can generate transient exceptions. Hence, multiple + * attempts are made to handle these transient exceptions. Returns 3-tuple + * (new files found, latest modification time among them, files with latest modification time) + */ + private def findNewFiles(currentTime: Long): (Seq[String], Long, Seq[String]) = { + logDebug("Trying to get new files for time " + currentTime) + var attempts = 0 + while (attempts < FileInputDStream.MAX_ATTEMPTS) { + attempts += 1 + try { + val filter = new CustomPathFilter(currentTime) + val newFiles = fs.listStatus(path, filter) + return (newFiles.map(_.getPath.toString), filter.latestModTime, filter.latestModTimeFiles.toSeq) + } catch { + case ioe: IOException => logWarning("Attempt " + attempts + " to get new files failed", ioe) + } + } + (Seq(), -1, Seq()) + } + /** Generate one RDD from an array of files */ - protected[streaming] def filesToRDD(files: Seq[String]): RDD[(K, V)] = { + private def filesToRDD(files: Seq[String]): RDD[(K, V)] = { new UnionRDD( context.sparkContext, files.map(file => context.sparkContext.newAPIHadoopFile[K, V, F](file)) @@ -189,10 +178,51 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K hadoopFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n") + "\n]" } } + + /** + * PathFilter to find new files that have modification timestamps <= current time, but have not + * been seen before (i.e. the file should not be in lastModTimeFiles) + * @param currentTime + */ + private[streaming] + class CustomPathFilter(currentTime: Long) extends PathFilter() { + // Latest file mod time seen in this round of fetching files and its corresponding files + var latestModTime = 0L + val latestModTimeFiles = new HashSet[String]() + + def accept(path: Path): Boolean = { + if (!filter(path)) { // Reject file if it does not satisfy filter + logDebug("Rejected by filter " + path) + return false + } else { // Accept file only if + val modTime = fs.getFileStatus(path).getModificationTime() + logDebug("Mod time for " + path + " is " + modTime) + if (modTime < prevModTime) { + logDebug("Mod time less than last mod time") + return false // If the file was created before the last time it was called + } else if (modTime == prevModTime && prevModTimeFiles.contains(path.toString)) { + logDebug("Mod time equal to last mod time, but file considered already") + return false // If the file was created exactly as lastModTime but not reported yet + } else if (modTime > currentTime) { + logDebug("Mod time more than valid time") + return false // If the file was created after the time this function call requires + } + if (modTime > latestModTime) { + latestModTime = modTime + latestModTimeFiles.clear() + logDebug("Latest mod time updated to " + latestModTime) + } + latestModTimeFiles += path.toString + logDebug("Accepted " + path) + return true + } + } + } } private[streaming] object FileInputDStream { + val MAX_ATTEMPTS = 10 def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".") } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index beb20831bd..e51754977c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -25,8 +25,10 @@ import org.scalatest.BeforeAndAfter import org.apache.commons.io.FileUtils import collection.mutable.{SynchronizedBuffer, ArrayBuffer} import util.{Clock, ManualClock} -import scala.util.Random import com.google.common.io.Files +import org.apache.hadoop.fs.{Path, FileSystem} +import org.apache.hadoop.conf.Configuration + /** @@ -44,7 +46,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { after { if (ssc != null) ssc.stop() - FileUtils.deleteDirectory(new File(checkpointDir)) + //FileUtils.deleteDirectory(new File(checkpointDir)) // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") @@ -66,7 +68,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") val stateStreamCheckpointInterval = Seconds(1) - + val fs = FileSystem.getLocal(new Configuration()) // this ensure checkpointing occurs at least once val firstNumBatches = (stateStreamCheckpointInterval / batchDuration).toLong * 2 val secondNumBatches = firstNumBatches @@ -90,11 +92,12 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { ssc.start() advanceTimeWithRealDelay(ssc, firstNumBatches) logInfo("Checkpoint data of state stream = \n" + stateStream.checkpointData) - assert(!stateStream.checkpointData.checkpointFiles.isEmpty, "No checkpointed RDDs in state stream before first failure") + assert(!stateStream.checkpointData.checkpointFiles.isEmpty, + "No checkpointed RDDs in state stream before first failure") stateStream.checkpointData.checkpointFiles.foreach { - case (time, data) => { - val file = new File(data.toString) - assert(file.exists(), "Checkpoint file '" + file +"' for time " + time + " for state stream before first failure does not exist") + case (time, file) => { + assert(fs.exists(new Path(file)), "Checkpoint file '" + file +"' for time " + time + + " for state stream before first failure does not exist") } } @@ -102,7 +105,8 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // and check whether the earlier checkpoint files are deleted val checkpointFiles = stateStream.checkpointData.checkpointFiles.map(x => new File(x._2)) advanceTimeWithRealDelay(ssc, secondNumBatches) - checkpointFiles.foreach(file => assert(!file.exists, "Checkpoint file '" + file + "' was not deleted")) + checkpointFiles.foreach(file => + assert(!file.exists, "Checkpoint file '" + file + "' was not deleted")) ssc.stop() // Restart stream computation using the checkpoint file and check whether @@ -110,19 +114,20 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { ssc = new StreamingContext(checkpointDir) stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head logInfo("Restored data of state stream = \n[" + stateStream.generatedRDDs.mkString("\n") + "]") - assert(!stateStream.generatedRDDs.isEmpty, "No restored RDDs in state stream after recovery from first failure") + assert(!stateStream.generatedRDDs.isEmpty, + "No restored RDDs in state stream after recovery from first failure") // Run one batch to generate a new checkpoint file and check whether some RDD // is present in the checkpoint data or not ssc.start() advanceTimeWithRealDelay(ssc, 1) - assert(!stateStream.checkpointData.checkpointFiles.isEmpty, "No checkpointed RDDs in state stream before second failure") + assert(!stateStream.checkpointData.checkpointFiles.isEmpty, + "No checkpointed RDDs in state stream before second failure") stateStream.checkpointData.checkpointFiles.foreach { - case (time, data) => { - val file = new File(data.toString) - assert(file.exists(), - "Checkpoint file '" + file +"' for time " + time + " for state stream before seconds failure does not exist") + case (time, file) => { + assert(fs.exists(new Path(file)), "Checkpoint file '" + file +"' for time " + time + + " for state stream before seconds failure does not exist") } } ssc.stop() @@ -132,7 +137,8 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { ssc = new StreamingContext(checkpointDir) stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head logInfo("Restored data of state stream = \n[" + stateStream.generatedRDDs.mkString("\n") + "]") - assert(!stateStream.generatedRDDs.isEmpty, "No restored RDDs in state stream after recovery from second failure") + assert(!stateStream.generatedRDDs.isEmpty, + "No restored RDDs in state stream after recovery from second failure") // Adjust manual clock time as if it is being restarted after a delay System.setProperty("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString) @@ -143,6 +149,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { ssc = null } + // This tests whether the systm can recover from a master failure with simple // non-stateful operations. This assumes as reliable, replayable input // source - TestInputDStream. @@ -191,6 +198,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { testCheckpointedOperation(input, operation, output, 7) } + // This tests whether file input stream remembers what files were seen before // the master failure and uses them again to process a large window operation. // It also tests whether batches, whose processing was incomplete due to the -- cgit v1.2.3 From 097e120c0c4132f007bfd0b0254b362ee9a02d8f Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 12 Dec 2013 20:41:51 -0800 Subject: Refactored streaming scheduler and added listener interface. - Refactored Scheduler + JobManager to JobGenerator + JobScheduler and added JobSet for cleaner code. Moved scheduler related code to streaming.scheduler package. - Added StreamingListener trait (similar to SparkListener) to enable gathering to streaming stats like processing times and delays. StreamingContext.addListener() to added listeners. - Deduped some code in streaming tests by modifying TestSuiteBase, and added StreamingListenerSuite. --- .../org/apache/spark/scheduler/SparkListener.scala | 2 +- .../org/apache/spark/streaming/Checkpoint.scala | 2 +- .../scala/org/apache/spark/streaming/DStream.scala | 11 +- .../org/apache/spark/streaming/DStreamGraph.scala | 1 + .../scala/org/apache/spark/streaming/Job.scala | 41 ----- .../org/apache/spark/streaming/JobManager.scala | 88 ----------- .../spark/streaming/NetworkInputTracker.scala | 174 -------------------- .../org/apache/spark/streaming/Scheduler.scala | 131 --------------- .../apache/spark/streaming/StreamingContext.scala | 17 +- .../spark/streaming/dstream/ForEachDStream.scala | 3 +- .../streaming/dstream/NetworkInputDStream.scala | 1 + .../spark/streaming/scheduler/BatchInfo.scala | 38 +++++ .../org/apache/spark/streaming/scheduler/Job.scala | 47 ++++++ .../spark/streaming/scheduler/JobGenerator.scala | 127 +++++++++++++++ .../spark/streaming/scheduler/JobScheduler.scala | 104 ++++++++++++ .../apache/spark/streaming/scheduler/JobSet.scala | 61 +++++++ .../streaming/scheduler/NetworkInputTracker.scala | 175 +++++++++++++++++++++ .../streaming/scheduler/StreamingListener.scala | 37 +++++ .../streaming/scheduler/StreamingListenerBus.scala | 81 ++++++++++ .../spark/streaming/BasicOperationsSuite.scala | 12 -- .../apache/spark/streaming/CheckpointSuite.scala | 26 ++- .../org/apache/spark/streaming/FailureSuite.scala | 13 +- .../apache/spark/streaming/InputStreamsSuite.scala | 12 -- .../spark/streaming/StreamingListenerSuite.scala | 71 +++++++++ .../org/apache/spark/streaming/TestSuiteBase.scala | 32 +++- .../spark/streaming/WindowOperationsSuite.scala | 14 +- 26 files changed, 811 insertions(+), 510 deletions(-) delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/Job.scala delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala (limited to 'core/src') 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 3841b5616d..2c5d87419d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -63,7 +63,7 @@ trait SparkListener { * Called when a task begins remotely fetching its result (will not be called for tasks that do * not need to fetch the result remotely). */ - def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { } + def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { } /** * Called when a task ends diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 9271914eb5..7b343d2376 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -40,7 +40,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) val graph = ssc.graph val checkpointDir = ssc.checkpointDir val checkpointDuration = ssc.checkpointDuration - val pendingTimes = ssc.scheduler.jobManager.getPendingTimes() + val pendingTimes = ssc.scheduler.getPendingTimes() val delaySeconds = MetadataCleaner.getDelaySeconds def validate() { 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 9ceff754c4..8001c49a76 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala @@ -17,23 +17,18 @@ package org.apache.spark.streaming -import org.apache.spark.streaming.dstream._ import StreamingContext._ -import org.apache.spark.util.MetadataCleaner - -//import Time._ - +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.scheduler.Job import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.MetadataCleaner -import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import java.io.{ObjectInputStream, IOException, ObjectOutputStream} -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.conf.Configuration /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index b9a58fded6..daed7ff7c3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -21,6 +21,7 @@ import dstream.InputDStream import java.io.{ObjectInputStream, IOException, ObjectOutputStream} import collection.mutable.ArrayBuffer import org.apache.spark.Logging +import org.apache.spark.streaming.scheduler.Job final private[streaming] class DStreamGraph extends Serializable with Logging { initLogging() diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Job.scala b/streaming/src/main/scala/org/apache/spark/streaming/Job.scala deleted file mode 100644 index 2128b7c7a6..0000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/Job.scala +++ /dev/null @@ -1,41 +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.spark.streaming - -import java.util.concurrent.atomic.AtomicLong - -private[streaming] -class Job(val time: Time, func: () => _) { - val id = Job.getNewId() - def run(): Long = { - val startTime = System.currentTimeMillis - func() - val stopTime = System.currentTimeMillis - (stopTime - startTime) - } - - override def toString = "streaming job " + id + " @ " + time -} - -private[streaming] -object Job { - val id = new AtomicLong(0) - - def getNewId() = id.getAndIncrement() -} - diff --git a/streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala deleted file mode 100644 index 5233129506..0000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala +++ /dev/null @@ -1,88 +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.spark.streaming - -import org.apache.spark.Logging -import org.apache.spark.SparkEnv -import java.util.concurrent.Executors -import collection.mutable.HashMap -import collection.mutable.ArrayBuffer - - -private[streaming] -class JobManager(ssc: StreamingContext, numThreads: Int = 1) extends Logging { - - class JobHandler(ssc: StreamingContext, job: Job) extends Runnable { - def run() { - SparkEnv.set(ssc.env) - try { - val timeTaken = job.run() - logInfo("Total delay: %.5f s for job %s of time %s (execution: %.5f s)".format( - (System.currentTimeMillis() - job.time.milliseconds) / 1000.0, job.id, job.time.milliseconds, timeTaken / 1000.0)) - } catch { - case e: Exception => - logError("Running " + job + " failed", e) - } - clearJob(job) - } - } - - initLogging() - - val jobExecutor = Executors.newFixedThreadPool(numThreads) - val jobs = new HashMap[Time, ArrayBuffer[Job]] - - def runJob(job: Job) { - jobs.synchronized { - jobs.getOrElseUpdate(job.time, new ArrayBuffer[Job]) += job - } - jobExecutor.execute(new JobHandler(ssc, job)) - logInfo("Added " + job + " to queue") - } - - def stop() { - jobExecutor.shutdown() - } - - private def clearJob(job: Job) { - var timeCleared = false - val time = job.time - jobs.synchronized { - val jobsOfTime = jobs.get(time) - if (jobsOfTime.isDefined) { - jobsOfTime.get -= job - if (jobsOfTime.get.isEmpty) { - jobs -= time - timeCleared = true - } - } else { - throw new Exception("Job finished for time " + job.time + - " but time does not exist in jobs") - } - } - if (timeCleared) { - ssc.scheduler.clearOldMetadata(time) - } - } - - def getPendingTimes(): Array[Time] = { - jobs.synchronized { - jobs.keySet.toArray - } - } -} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala deleted file mode 100644 index b97fb7e6e3..0000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala +++ /dev/null @@ -1,174 +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.spark.streaming - -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 - -import akka.actor._ -import akka.pattern.ask -import akka.util.duration._ -import akka.dispatch._ -import org.apache.spark.storage.BlockId - -private[streaming] sealed trait NetworkInputTrackerMessage -private[streaming] case class RegisterReceiver(streamId: Int, receiverActor: ActorRef) extends NetworkInputTrackerMessage -private[streaming] case class AddBlocks(streamId: Int, blockIds: Seq[BlockId], metadata: Any) extends NetworkInputTrackerMessage -private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) extends NetworkInputTrackerMessage - -/** - * This class manages the execution of the receivers of NetworkInputDStreams. - */ -private[streaming] -class NetworkInputTracker( - @transient ssc: StreamingContext, - @transient networkInputStreams: Array[NetworkInputDStream[_]]) - extends Logging { - - val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*) - val receiverExecutor = new ReceiverExecutor() - val receiverInfo = new HashMap[Int, ActorRef] - val receivedBlockIds = new HashMap[Int, Queue[BlockId]] - val timeout = 5000.milliseconds - - var currentTime: Time = null - - /** Start the actor and receiver execution thread. */ - def start() { - ssc.env.actorSystem.actorOf(Props(new NetworkInputTrackerActor), "NetworkInputTracker") - receiverExecutor.start() - } - - /** Stop the receiver execution thread. */ - def stop() { - // TODO: stop the actor as well - receiverExecutor.interrupt() - receiverExecutor.stopReceivers() - } - - /** Return all the blocks received from a receiver. */ - def getBlockIds(receiverId: Int, time: Time): Array[BlockId] = synchronized { - val queue = receivedBlockIds.synchronized { - receivedBlockIds.getOrElse(receiverId, new Queue[BlockId]()) - } - val result = queue.synchronized { - queue.dequeueAll(x => true) - } - logInfo("Stream " + receiverId + " received " + result.size + " blocks") - result.toArray - } - - /** Actor to receive messages from the receivers. */ - private class NetworkInputTrackerActor extends Actor { - def receive = { - case RegisterReceiver(streamId, receiverActor) => { - if (!networkInputStreamMap.contains(streamId)) { - throw new Exception("Register received for unexpected id " + streamId) - } - receiverInfo += ((streamId, receiverActor)) - logInfo("Registered receiver for network stream " + streamId + " from " + sender.path.address) - sender ! true - } - case AddBlocks(streamId, blockIds, metadata) => { - val tmp = receivedBlockIds.synchronized { - if (!receivedBlockIds.contains(streamId)) { - receivedBlockIds += ((streamId, new Queue[BlockId])) - } - receivedBlockIds(streamId) - } - tmp.synchronized { - tmp ++= blockIds - } - networkInputStreamMap(streamId).addMetadata(metadata) - } - case DeregisterReceiver(streamId, msg) => { - receiverInfo -= streamId - logError("De-registered receiver for network stream " + streamId - + " with message " + msg) - //TODO: Do something about the corresponding NetworkInputDStream - } - } - } - - /** This thread class runs all the receivers on the cluster. */ - class ReceiverExecutor extends Thread { - val env = ssc.env - - override def run() { - try { - SparkEnv.set(env) - startReceivers() - } catch { - case ie: InterruptedException => logInfo("ReceiverExecutor interrupted") - } finally { - stopReceivers() - } - } - - /** - * Get the receivers from the NetworkInputDStreams, distributes them to the - * worker nodes as a parallel collection, and runs them. - */ - def startReceivers() { - val receivers = networkInputStreams.map(nis => { - val rcvr = nis.getReceiver() - rcvr.setStreamId(nis.id) - rcvr - }) - - // Right now, we only honor preferences if all receivers have them - val hasLocationPreferences = receivers.map(_.getLocationPreference().isDefined).reduce(_ && _) - - // Create the parallel collection of receivers to distributed them on the worker nodes - val tempRDD = - if (hasLocationPreferences) { - val receiversWithPreferences = receivers.map(r => (r, Seq(r.getLocationPreference().toString))) - ssc.sc.makeRDD[NetworkReceiver[_]](receiversWithPreferences) - } - else { - ssc.sc.makeRDD(receivers, receivers.size) - } - - // Function to start the receiver on the worker node - val startReceiver = (iterator: Iterator[NetworkReceiver[_]]) => { - if (!iterator.hasNext) { - throw new Exception("Could not start receiver as details not found.") - } - iterator.next().start() - } - // Run the dummy Spark job to ensure that all slaves have registered. - // This avoids all the receivers to be scheduled on the same node. - ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect() - - // Distribute the receivers and start them - ssc.sparkContext.runJob(tempRDD, startReceiver) - } - - /** Stops the receivers. */ - def stopReceivers() { - // Signal the receivers to stop - receiverInfo.values.foreach(_ ! StopReceiver) - } - } -} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala deleted file mode 100644 index ed892e33e6..0000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala +++ /dev/null @@ -1,131 +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.spark.streaming - -import util.{ManualClock, RecurringTimer, Clock} -import org.apache.spark.SparkEnv -import org.apache.spark.Logging - -private[streaming] -class Scheduler(ssc: StreamingContext) extends Logging { - - initLogging() - - val concurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt - val jobManager = new JobManager(ssc, concurrentJobs) - val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) { - new CheckpointWriter(ssc.checkpointDir) - } else { - null - } - - 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))) - val graph = ssc.graph - var latestTime: Time = null - - def start() = synchronized { - if (ssc.isCheckpointPresent) { - restart() - } else { - startFirstTime() - } - logInfo("Scheduler started") - } - - def stop() = synchronized { - timer.stop() - jobManager.stop() - if (checkpointWriter != null) checkpointWriter.stop() - ssc.graph.stop() - logInfo("Scheduler stopped") - } - - private def startFirstTime() { - val startTime = new Time(timer.getStartTime()) - graph.start(startTime - graph.batchDuration) - timer.start(startTime.milliseconds) - logInfo("Scheduler's timer started at " + startTime) - } - - private def restart() { - - // If manual clock is being used for testing, then - // either set the manual clock to the last checkpointed time, - // or if the property is defined set it to that time - if (clock.isInstanceOf[ManualClock]) { - val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds - val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong - clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime) - } - - val batchDuration = ssc.graph.batchDuration - - // Batches when the master was down, that is, - // between the checkpoint and current restart time - val checkpointTime = ssc.initialCheckpoint.checkpointTime - val restartTime = new Time(timer.getRestartTime(graph.zeroTime.milliseconds)) - val downTimes = checkpointTime.until(restartTime, batchDuration) - logInfo("Batches during down time: " + downTimes.mkString(", ")) - - // Batches that were unprocessed before failure - val pendingTimes = ssc.initialCheckpoint.pendingTimes - logInfo("Batches pending processing: " + pendingTimes.mkString(", ")) - // Reschedule jobs for these times - val timesToReschedule = (pendingTimes ++ downTimes).distinct.sorted(Time.ordering) - logInfo("Batches to reschedule: " + timesToReschedule.mkString(", ")) - timesToReschedule.foreach(time => - graph.generateJobs(time).foreach(jobManager.runJob) - ) - - // Restart the timer - timer.start(restartTime.milliseconds) - logInfo("Scheduler's timer restarted at " + restartTime) - } - - /** Generate jobs and perform checkpoint for the given `time`. */ - def generateJobs(time: Time) { - SparkEnv.set(ssc.env) - logInfo("\n-----------------------------------------------------\n") - graph.generateJobs(time).foreach(jobManager.runJob) - latestTime = time - doCheckpoint(time) - } - - /** - * Clear old metadata assuming jobs of `time` have finished processing. - * And also perform checkpoint. - */ - def clearOldMetadata(time: Time) { - ssc.graph.clearOldMetadata(time) - doCheckpoint(time) - } - - /** Perform checkpoint for the give `time`. */ - def doCheckpoint(time: Time) = synchronized { - if (ssc.checkpointDuration != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) { - logInfo("Checkpointing graph for time " + time) - ssc.graph.updateCheckpointData(time) - checkpointWriter.write(new Checkpoint(ssc, time)) - } - } -} - 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 70bf902143..83f1cadb48 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -46,6 +46,7 @@ import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path import twitter4j.Status import twitter4j.auth.Authorization +import org.apache.spark.streaming.scheduler._ /** @@ -146,9 +147,10 @@ class StreamingContext private ( } } - protected[streaming] var checkpointDuration: Duration = if (isCheckpointPresent) cp_.checkpointDuration else null - protected[streaming] var receiverJobThread: Thread = null - protected[streaming] var scheduler: Scheduler = null + protected[streaming] val checkpointDuration: Duration = { + if (isCheckpointPresent) cp_.checkpointDuration else graph.batchDuration + } + protected[streaming] val scheduler = new JobScheduler(this) /** * Return the associated Spark context @@ -510,6 +512,10 @@ class StreamingContext private ( graph.addOutputStream(outputStream) } + def addListener(streamingListener: StreamingListener) { + scheduler.listenerBus.addListener(streamingListener) + } + protected def validate() { assert(graph != null, "Graph is null") graph.validate() @@ -525,9 +531,6 @@ class StreamingContext private ( * Start the execution of the streams. */ def start() { - if (checkpointDir != null && checkpointDuration == null && graph != null) { - checkpointDuration = graph.batchDuration - } validate() @@ -545,7 +548,6 @@ class StreamingContext private ( Thread.sleep(1000) // Start the scheduler - scheduler = new Scheduler(this) scheduler.start() } @@ -556,7 +558,6 @@ class StreamingContext private ( try { if (scheduler != null) scheduler.stop() if (networkInputTracker != null) networkInputTracker.stop() - if (receiverJobThread != null) receiverJobThread.interrupt() sc.stop() logInfo("StreamingContext stopped successfully") } catch { 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 e21bac4602..0072248b7d 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 @@ -18,7 +18,8 @@ package org.apache.spark.streaming.dstream import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.{Duration, DStream, Job, Time} +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.streaming.scheduler.Job private[streaming] class ForEachDStream[T: 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 a82862c802..1df7f547c9 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 @@ -32,6 +32,7 @@ import org.apache.spark.streaming._ import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.rdd.{RDD, BlockRDD} import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId} +import org.apache.spark.streaming.scheduler.{DeregisterReceiver, AddBlocks, RegisterReceiver} /** * 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/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala new file mode 100644 index 0000000000..798598ad50 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streaming.scheduler + +import org.apache.spark.streaming.Time + +case class BatchInfo( + batchTime: Time, + submissionTime: Long, + processingStartTime: Option[Long], + processingEndTime: Option[Long] + ) { + + def schedulingDelay = processingStartTime.map(_ - submissionTime) + + def processingDelay = processingEndTime.zip(processingStartTime).map(x => x._1 - x._2).headOption + + def totalDelay = schedulingDelay.zip(processingDelay).map(x => x._1 + x._2).headOption +} + + + + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala new file mode 100644 index 0000000000..bca5e1f1a5 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streaming.scheduler + +import java.util.concurrent.atomic.AtomicLong +import org.apache.spark.streaming.Time + +private[streaming] +class Job(val time: Time, func: () => _) { + var id: String = _ + + def run(): Long = { + val startTime = System.currentTimeMillis + func() + val stopTime = System.currentTimeMillis + (stopTime - startTime) + } + + def setId(number: Int) { + id = "streaming job " + time + "." + number + } + + override def toString = id +} +/* +private[streaming] +object Job { + val id = new AtomicLong(0) + + def getNewId() = id.getAndIncrement() +} +*/ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala new file mode 100644 index 0000000000..5d3ce9c398 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streaming.scheduler + +import org.apache.spark.SparkEnv +import org.apache.spark.Logging +import org.apache.spark.streaming.{Checkpoint, Time, CheckpointWriter} +import org.apache.spark.streaming.util.{ManualClock, RecurringTimer, Clock} + +private[streaming] +class JobGenerator(jobScheduler: JobScheduler) extends Logging { + + initLogging() + val ssc = jobScheduler.ssc + 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))) + val graph = ssc.graph + lazy val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) { + new CheckpointWriter(ssc.checkpointDir) + } else { + null + } + + var latestTime: Time = null + + def start() = synchronized { + if (ssc.isCheckpointPresent) { + restart() + } else { + startFirstTime() + } + logInfo("JobGenerator started") + } + + def stop() = synchronized { + timer.stop() + if (checkpointWriter != null) checkpointWriter.stop() + ssc.graph.stop() + logInfo("JobGenerator stopped") + } + + private def startFirstTime() { + val startTime = new Time(timer.getStartTime()) + graph.start(startTime - graph.batchDuration) + timer.start(startTime.milliseconds) + logInfo("JobGenerator's timer started at " + startTime) + } + + private def restart() { + // If manual clock is being used for testing, then + // either set the manual clock to the last checkpointed time, + // or if the property is defined set it to that time + if (clock.isInstanceOf[ManualClock]) { + val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds + val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong + clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime) + } + + val batchDuration = ssc.graph.batchDuration + + // Batches when the master was down, that is, + // between the checkpoint and current restart time + val checkpointTime = ssc.initialCheckpoint.checkpointTime + val restartTime = new Time(timer.getRestartTime(graph.zeroTime.milliseconds)) + val downTimes = checkpointTime.until(restartTime, batchDuration) + logInfo("Batches during down time: " + downTimes.mkString(", ")) + + // Batches that were unprocessed before failure + val pendingTimes = ssc.initialCheckpoint.pendingTimes + logInfo("Batches pending processing: " + pendingTimes.mkString(", ")) + // Reschedule jobs for these times + val timesToReschedule = (pendingTimes ++ downTimes).distinct.sorted(Time.ordering) + logInfo("Batches to reschedule: " + timesToReschedule.mkString(", ")) + timesToReschedule.foreach(time => + jobScheduler.runJobs(time, graph.generateJobs(time)) + ) + + // Restart the timer + timer.start(restartTime.milliseconds) + logInfo("JobGenerator's timer restarted at " + restartTime) + } + + /** Generate jobs and perform checkpoint for the given `time`. */ + private def generateJobs(time: Time) { + SparkEnv.set(ssc.env) + logInfo("\n-----------------------------------------------------\n") + jobScheduler.runJobs(time, graph.generateJobs(time)) + latestTime = time + doCheckpoint(time) + } + + /** + * On batch completion, clear old metadata and checkpoint computation. + */ + private[streaming] def onBatchCompletion(time: Time) { + ssc.graph.clearOldMetadata(time) + doCheckpoint(time) + } + + /** Perform checkpoint for the give `time`. */ + private def doCheckpoint(time: Time) = synchronized { + if (checkpointWriter != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) { + logInfo("Checkpointing graph for time " + time) + ssc.graph.updateCheckpointData(time) + checkpointWriter.write(new Checkpoint(ssc, time)) + } + } +} + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala new file mode 100644 index 0000000000..14906fd720 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streaming.scheduler + +import org.apache.spark.Logging +import org.apache.spark.SparkEnv +import java.util.concurrent.{TimeUnit, ConcurrentHashMap, Executors} +import scala.collection.mutable.HashSet +import org.apache.spark.streaming._ + + +private[streaming] +class JobScheduler(val ssc: StreamingContext) extends Logging { + + initLogging() + + val jobSets = new ConcurrentHashMap[Time, JobSet] + val numConcurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt + val executor = Executors.newFixedThreadPool(numConcurrentJobs) + val generator = new JobGenerator(this) + val listenerBus = new StreamingListenerBus() + + def clock = generator.clock + + def start() { + generator.start() + } + + def stop() { + generator.stop() + executor.shutdown() + if (!executor.awaitTermination(5, TimeUnit.SECONDS)) { + executor.shutdownNow() + } + } + + def runJobs(time: Time, jobs: Seq[Job]) { + if (jobs.isEmpty) { + logInfo("No jobs added for time " + time) + } else { + val jobSet = new JobSet(time, jobs) + jobSets.put(time, jobSet) + jobSet.jobs.foreach(job => executor.execute(new JobHandler(job))) + logInfo("Added jobs for time " + time) + } + } + + def getPendingTimes(): Array[Time] = { + jobSets.keySet.toArray(new Array[Time](0)) + } + + private def beforeJobStart(job: Job) { + val jobSet = jobSets.get(job.time) + if (!jobSet.hasStarted) { + listenerBus.post(StreamingListenerBatchStarted(jobSet.toBatchInfo())) + } + jobSet.beforeJobStart(job) + logInfo("Starting job " + job.id + " from job set of time " + jobSet.time) + SparkEnv.set(generator.ssc.env) + } + + private def afterJobEnd(job: Job) { + val jobSet = jobSets.get(job.time) + jobSet.afterJobStop(job) + logInfo("Finished job " + job.id + " from job set of time " + jobSet.time) + if (jobSet.hasCompleted) { + listenerBus.post(StreamingListenerBatchCompleted(jobSet.toBatchInfo())) + jobSets.remove(jobSet.time) + generator.onBatchCompletion(jobSet.time) + logInfo("Total delay: %.3f s for time %s (execution: %.3f s)".format( + jobSet.totalDelay / 1000.0, jobSet.time.toString, + jobSet.processingDelay / 1000.0 + )) + } + } + + class JobHandler(job: Job) extends Runnable { + def run() { + beforeJobStart(job) + try { + job.run() + } catch { + case e: Exception => + logError("Running " + job + " failed", e) + } + afterJobEnd(job) + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala new file mode 100644 index 0000000000..05233d095b --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streaming.scheduler + +import scala.collection.mutable.HashSet +import org.apache.spark.streaming.Time + +private[streaming] +case class JobSet(time: Time, jobs: Seq[Job]) { + + private val incompleteJobs = new HashSet[Job]() + var submissionTime = System.currentTimeMillis() + var processingStartTime = -1L + var processingEndTime = -1L + + jobs.zipWithIndex.foreach { case (job, i) => job.setId(i) } + incompleteJobs ++= jobs + + def beforeJobStart(job: Job) { + if (processingStartTime < 0) processingStartTime = System.currentTimeMillis() + } + + def afterJobStop(job: Job) { + incompleteJobs -= job + if (hasCompleted) processingEndTime = System.currentTimeMillis() + } + + def hasStarted() = (processingStartTime > 0) + + def hasCompleted() = incompleteJobs.isEmpty + + def processingDelay = processingEndTime - processingStartTime + + def totalDelay = { + processingEndTime - time.milliseconds + } + + def toBatchInfo(): BatchInfo = { + new BatchInfo( + time, + submissionTime, + if (processingStartTime >= 0 ) Some(processingStartTime) else None, + if (processingEndTime >= 0 ) Some(processingEndTime) else None + ) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala new file mode 100644 index 0000000000..c759302a61 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streaming.scheduler + +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 + +import akka.actor._ +import akka.pattern.ask +import akka.util.duration._ +import akka.dispatch._ +import org.apache.spark.storage.BlockId +import org.apache.spark.streaming.{Time, StreamingContext} + +private[streaming] sealed trait NetworkInputTrackerMessage +private[streaming] case class RegisterReceiver(streamId: Int, receiverActor: ActorRef) extends NetworkInputTrackerMessage +private[streaming] case class AddBlocks(streamId: Int, blockIds: Seq[BlockId], metadata: Any) extends NetworkInputTrackerMessage +private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) extends NetworkInputTrackerMessage + +/** + * This class manages the execution of the receivers of NetworkInputDStreams. + */ +private[streaming] +class NetworkInputTracker( + @transient ssc: StreamingContext, + @transient networkInputStreams: Array[NetworkInputDStream[_]]) + extends Logging { + + val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*) + val receiverExecutor = new ReceiverExecutor() + val receiverInfo = new HashMap[Int, ActorRef] + val receivedBlockIds = new HashMap[Int, Queue[BlockId]] + val timeout = 5000.milliseconds + + var currentTime: Time = null + + /** Start the actor and receiver execution thread. */ + def start() { + ssc.env.actorSystem.actorOf(Props(new NetworkInputTrackerActor), "NetworkInputTracker") + receiverExecutor.start() + } + + /** Stop the receiver execution thread. */ + def stop() { + // TODO: stop the actor as well + receiverExecutor.interrupt() + receiverExecutor.stopReceivers() + } + + /** Return all the blocks received from a receiver. */ + def getBlockIds(receiverId: Int, time: Time): Array[BlockId] = synchronized { + val queue = receivedBlockIds.synchronized { + receivedBlockIds.getOrElse(receiverId, new Queue[BlockId]()) + } + val result = queue.synchronized { + queue.dequeueAll(x => true) + } + logInfo("Stream " + receiverId + " received " + result.size + " blocks") + result.toArray + } + + /** Actor to receive messages from the receivers. */ + private class NetworkInputTrackerActor extends Actor { + def receive = { + case RegisterReceiver(streamId, receiverActor) => { + if (!networkInputStreamMap.contains(streamId)) { + throw new Exception("Register received for unexpected id " + streamId) + } + receiverInfo += ((streamId, receiverActor)) + logInfo("Registered receiver for network stream " + streamId + " from " + sender.path.address) + sender ! true + } + case AddBlocks(streamId, blockIds, metadata) => { + val tmp = receivedBlockIds.synchronized { + if (!receivedBlockIds.contains(streamId)) { + receivedBlockIds += ((streamId, new Queue[BlockId])) + } + receivedBlockIds(streamId) + } + tmp.synchronized { + tmp ++= blockIds + } + networkInputStreamMap(streamId).addMetadata(metadata) + } + case DeregisterReceiver(streamId, msg) => { + receiverInfo -= streamId + logError("De-registered receiver for network stream " + streamId + + " with message " + msg) + //TODO: Do something about the corresponding NetworkInputDStream + } + } + } + + /** This thread class runs all the receivers on the cluster. */ + class ReceiverExecutor extends Thread { + val env = ssc.env + + override def run() { + try { + SparkEnv.set(env) + startReceivers() + } catch { + case ie: InterruptedException => logInfo("ReceiverExecutor interrupted") + } finally { + stopReceivers() + } + } + + /** + * Get the receivers from the NetworkInputDStreams, distributes them to the + * worker nodes as a parallel collection, and runs them. + */ + def startReceivers() { + val receivers = networkInputStreams.map(nis => { + val rcvr = nis.getReceiver() + rcvr.setStreamId(nis.id) + rcvr + }) + + // Right now, we only honor preferences if all receivers have them + val hasLocationPreferences = receivers.map(_.getLocationPreference().isDefined).reduce(_ && _) + + // Create the parallel collection of receivers to distributed them on the worker nodes + val tempRDD = + if (hasLocationPreferences) { + val receiversWithPreferences = receivers.map(r => (r, Seq(r.getLocationPreference().toString))) + ssc.sc.makeRDD[NetworkReceiver[_]](receiversWithPreferences) + } + else { + ssc.sc.makeRDD(receivers, receivers.size) + } + + // Function to start the receiver on the worker node + val startReceiver = (iterator: Iterator[NetworkReceiver[_]]) => { + if (!iterator.hasNext) { + throw new Exception("Could not start receiver as details not found.") + } + iterator.next().start() + } + // Run the dummy Spark job to ensure that all slaves have registered. + // This avoids all the receivers to be scheduled on the same node. + ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect() + + // Distribute the receivers and start them + ssc.sparkContext.runJob(tempRDD, startReceiver) + } + + /** Stops the receivers. */ + def stopReceivers() { + // Signal the receivers to stop + receiverInfo.values.foreach(_ ! StopReceiver) + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala new file mode 100644 index 0000000000..49fd0d29c3 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streaming.scheduler + +sealed trait StreamingListenerEvent + +case class StreamingListenerBatchCompleted(batchInfo: BatchInfo) extends StreamingListenerEvent + +case class StreamingListenerBatchStarted(batchInfo: BatchInfo) extends StreamingListenerEvent + +trait StreamingListener { + + /** + * Called when processing of a batch has completed + */ + def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { } + + /** + * Called when processing of a batch has started + */ + def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { } +} \ No newline at end of file diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala new file mode 100644 index 0000000000..324e491914 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streaming.scheduler + +import org.apache.spark.Logging +import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import java.util.concurrent.LinkedBlockingQueue + +/** Asynchronously passes StreamingListenerEvents to registered StreamingListeners. */ +private[spark] class StreamingListenerBus() extends Logging { + private val listeners = new ArrayBuffer[StreamingListener]() with SynchronizedBuffer[StreamingListener] + + /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than + * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ + private val EVENT_QUEUE_CAPACITY = 10000 + private val eventQueue = new LinkedBlockingQueue[StreamingListenerEvent](EVENT_QUEUE_CAPACITY) + private var queueFullErrorMessageLogged = false + + new Thread("StreamingListenerBus") { + setDaemon(true) + override def run() { + while (true) { + val event = eventQueue.take + event match { + case batchStarted: StreamingListenerBatchStarted => + listeners.foreach(_.onBatchStarted(batchStarted)) + case batchCompleted: StreamingListenerBatchCompleted => + listeners.foreach(_.onBatchCompleted(batchCompleted)) + case _ => + } + } + } + }.start() + + def addListener(listener: StreamingListener) { + listeners += listener + } + + def post(event: StreamingListenerEvent) { + val eventAdded = eventQueue.offer(event) + if (!eventAdded && !queueFullErrorMessageLogged) { + logError("Dropping SparkListenerEvent because no remaining room in event queue. " + + "This likely means one of the SparkListeners is too slow and cannot keep up with the " + + "rate at which tasks are being started by the scheduler.") + queueFullErrorMessageLogged = true + } + } + + /** + * Waits until there are no more events in the queue, or until the specified time has elapsed. + * Used for testing only. Returns true if the queue has emptied and false is the specified time + * elapsed before the queue emptied. + */ + def waitUntilEmpty(timeoutMillis: Int): Boolean = { + val finishTime = System.currentTimeMillis + timeoutMillis + while (!eventQueue.isEmpty()) { + if (System.currentTimeMillis > finishTime) { + return false + } + /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify + * add overhead in the general case. */ + Thread.sleep(10) + } + return true + } +} \ No newline at end of file diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 259ef1608c..b35ca00b53 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -26,18 +26,6 @@ import util.ManualClock class BasicOperationsSuite extends TestSuiteBase { - override def framework() = "BasicOperationsSuite" - - before { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") - } - - after { - // 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") - } - test("map") { val input = Seq(1 to 4, 5 to 8, 9 to 12) testOperation( diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index beb20831bd..c93075e3b3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -34,31 +34,25 @@ import com.google.common.io.Files * the checkpointing of a DStream's RDDs as well as the checkpointing of * the whole DStream graph. */ -class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { +class CheckpointSuite extends TestSuiteBase { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + var ssc: StreamingContext = null + + override def batchDuration = Milliseconds(500) + + override def actuallyWait = true // to allow checkpoints to be written - before { + override def beforeFunction() { + super.beforeFunction() FileUtils.deleteDirectory(new File(checkpointDir)) } - after { + override def afterFunction() { + super.afterFunction() if (ssc != null) ssc.stop() FileUtils.deleteDirectory(new File(checkpointDir)) - - // 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") } - var ssc: StreamingContext = null - - override def framework = "CheckpointSuite" - - override def batchDuration = Milliseconds(500) - - override def actuallyWait = true - test("basic rdd checkpoints + dstream graph checkpoint recovery") { assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala index 6337c5359c..da9b04de1a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala @@ -32,17 +32,22 @@ import collection.mutable.ArrayBuffer * This testsuite tests master failures at random times while the stream is running using * the real clock. */ -class FailureSuite extends FunSuite with BeforeAndAfter with Logging { +class FailureSuite extends TestSuiteBase with Logging { var directory = "FailureSuite" val numBatches = 30 - val batchDuration = Milliseconds(1000) - before { + override def batchDuration = Milliseconds(1000) + + override def useManualClock = false + + override def beforeFunction() { + super.beforeFunction() FileUtils.deleteDirectory(new File(directory)) } - after { + override def afterFunction() { + super.afterFunction() FileUtils.deleteDirectory(new File(directory)) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 7dc82decef..62a9f120b4 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -50,18 +50,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val testPort = 9999 - override def checkpointDir = "checkpoint" - - before { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") - } - - after { - // 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") - } - test("socket input stream") { // Start the server val testServer = new TestServer() diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala new file mode 100644 index 0000000000..826c839932 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.streaming + +import org.apache.spark.streaming.scheduler._ +import scala.collection.mutable.ArrayBuffer +import org.scalatest.matchers.ShouldMatchers + +class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers{ + + val input = (1 to 4).map(Seq(_)).toSeq + val operation = (d: DStream[Int]) => d.map(x => x) + + // To make sure that the processing start and end times in collected + // information are different for successive batches + override def batchDuration = Milliseconds(100) + override def actuallyWait = true + + test("basic BatchInfo generation") { + val ssc = setupStreams(input, operation) + val collector = new BatchInfoCollector + ssc.addListener(collector) + runStreams(ssc, input.size, input.size) + val batchInfos = collector.batchInfos + batchInfos should have size 4 + + batchInfos.foreach(info => { + info.schedulingDelay should not be None + info.processingDelay should not be None + info.totalDelay should not be None + info.schedulingDelay.get should be >= 0L + info.processingDelay.get should be >= 0L + info.totalDelay.get should be >= 0L + }) + + isInIncreasingOrder(batchInfos.map(_.submissionTime)) should be (true) + isInIncreasingOrder(batchInfos.map(_.processingStartTime.get)) should be (true) + isInIncreasingOrder(batchInfos.map(_.processingEndTime.get)) should be (true) + } + + /** Check if a sequence of numbers is in increasing order */ + def isInIncreasingOrder(seq: Seq[Long]): Boolean = { + for(i <- 1 until seq.size) { + if (seq(i - 1) > seq(i)) return false + } + true + } + + /** Listener that collects information on processed batches */ + class BatchInfoCollector extends StreamingListener { + val batchInfos = new ArrayBuffer[BatchInfo] + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { + batchInfos += batchCompleted.batchInfo + } + } +} 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 8c8c359e6e..fbbeb8f0ee 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -109,7 +109,7 @@ class TestOutputStreamWithPartitions[T: ClassManifest](parent: DStream[T], trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Name of the framework for Spark context - def framework = "TestSuiteBase" + def framework = this.getClass.getSimpleName // Master for Spark context def master = "local[2]" @@ -126,9 +126,39 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Maximum time to wait before the test times out def maxWaitTimeMillis = 10000 + // Whether to use manual clock or not + def useManualClock = true + // Whether to actually wait in real time before changing manual clock def actuallyWait = false + // Default before function for any streaming test suite. Override this + // if you want to add your stuff to "before" (i.e., don't call before { } ) + def beforeFunction() { + if (useManualClock) { + System.setProperty( + "spark.streaming.clock", + "org.apache.spark.streaming.util.ManualClock" + ) + } else { + System.clearProperty("spark.streaming.clock") + } + // 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") + } + + // Default after function for any streaming test suite. Override this + // if you want to add your stuff to "after" (i.e., don't call after { } ) + def afterFunction() { + // 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") + } + + before(beforeFunction) + after(afterFunction) + /** * Set up required DStreams to test the DStream operation using the two sequences * of input collections. diff --git a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala index f50e05c0d8..6b4aaefcdf 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala @@ -22,19 +22,9 @@ import collection.mutable.ArrayBuffer class WindowOperationsSuite extends TestSuiteBase { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + override def maxWaitTimeMillis = 20000 // large window tests can sometimes take longer - override def framework = "WindowOperationsSuite" - - override def maxWaitTimeMillis = 20000 - - override def batchDuration = Seconds(1) - - after { - // 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") - } + override def batchDuration = Seconds(1) // making sure its visible in this class val largerSlideInput = Seq( Seq(("a", 1)), -- cgit v1.2.3 From a854cc536d79fc2c503d70bf8b6999d74f946876 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 13 Dec 2013 15:19:51 +0530 Subject: Review comments on the PR for scala 2.10 migration. --- .../main/scala/org/apache/spark/SparkContext.scala | 3 +-- .../org/apache/spark/api/java/JavaPairRDD.scala | 22 ++++++++-------------- .../org/apache/spark/api/java/JavaRDDLike.scala | 9 +++------ .../apache/spark/api/java/JavaSparkContext.scala | 15 +++++---------- .../spark/api/java/function/FlatMapFunction.scala | 2 +- .../org/apache/spark/api/python/PythonRDD.scala | 4 ++-- .../org/apache/spark/deploy/ExecutorState.scala | 3 +-- .../org/apache/spark/deploy/client/Client.scala | 2 +- .../scala/org/apache/spark/rdd/CartesianRDD.scala | 2 +- .../main/scala/org/apache/spark/rdd/EmptyRDD.scala | 2 +- .../org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- .../org/apache/spark/scheduler/TaskLocality.scala | 4 +--- .../org/apache/spark/storage/BlockManager.scala | 1 - project/SparkBuild.scala | 6 +++--- .../org/apache/spark/repl/SparkExprTyper.scala | 1 - 15 files changed, 29 insertions(+), 49 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f3ce4c879d..a0f794edfd 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -26,8 +26,7 @@ import scala.collection.Map import scala.collection.generic.Growable import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import scala.reflect.{ ClassTag, classTag} -import scala.util.DynamicVariable +import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path 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 eeea0eddb1..1167c12022 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 @@ -158,8 +158,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K mergeValue: JFunction2[C, V, C], mergeCombiners: JFunction2[C, C, C], partitioner: Partitioner): JavaPairRDD[K, C] = { - implicit val cm: ClassTag[C] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] + implicit val cm: ClassTag[C] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] fromRDD(rdd.combineByKey( createCombiner, mergeValue, @@ -323,8 +322,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K def combineByKey[C](createCombiner: JFunction[V, C], mergeValue: JFunction2[C, V, C], mergeCombiners: JFunction2[C, C, C]): JavaPairRDD[K, C] = { - implicit val cm: ClassTag[C] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] + implicit val cm: ClassTag[C] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] fromRDD(combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(rdd))) } @@ -415,8 +413,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K * this also retains the original RDD's partitioning. */ def mapValues[U](f: JFunction[V, U]): JavaPairRDD[K, U] = { - implicit val cm: ClassTag[U] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] + implicit val cm: ClassTag[U] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] fromRDD(rdd.mapValues(f)) } @@ -427,8 +424,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairRDD[K, U] = { import scala.collection.JavaConverters._ def fn = (x: V) => f.apply(x).asScala - implicit val cm: ClassTag[U] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] + implicit val cm: ClassTag[U] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] fromRDD(rdd.flatMapValues(fn)) } @@ -609,8 +605,8 @@ object JavaPairRDD { rddToPairRDDFunctions(rdd).mapValues(seqAsJavaList _) def cogroupResultToJava[W, K, V](rdd: RDD[(K, (Seq[V], Seq[W]))])(implicit kcm: ClassTag[K], - vcm: ClassTag[V]): RDD[(K, (JList[V], JList[W]))] = rddToPairRDDFunctions(rdd).mapValues((x: (Seq[V], - Seq[W])) => (seqAsJavaList(x._1), seqAsJavaList(x._2))) + vcm: ClassTag[V]): RDD[(K, (JList[V], JList[W]))] = rddToPairRDDFunctions(rdd) + .mapValues((x: (Seq[V], Seq[W])) => (seqAsJavaList(x._1), seqAsJavaList(x._2))) def cogroupResult2ToJava[W1, W2, K, V](rdd: RDD[(K, (Seq[V], Seq[W1], Seq[W2]))])(implicit kcm: ClassTag[K]) : RDD[(K, (JList[V], JList[W1], @@ -627,10 +623,8 @@ object JavaPairRDD { /** Convert a JavaRDD of key-value pairs to JavaPairRDD. */ def fromJavaRDD[K, V](rdd: JavaRDD[(K, V)]): JavaPairRDD[K, V] = { - implicit val cmk: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val cmv: ClassTag[V] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + implicit val cmk: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val cmv: ClassTag[V] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] new JavaPairRDD[K, V](rdd.rdd) } 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 c9c13f7a26..9e912d3adb 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 @@ -159,16 +159,14 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * elements (a, b) where a is in `this` and b is in `other`. */ def cartesian[U](other: JavaRDDLike[U, _]): JavaPairRDD[T, U] = - JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classTag))(classTag, - other.classTag) + JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classTag))(classTag, other.classTag) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JList[T]] = { - implicit val kcm: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] implicit val vcm: ClassTag[JList[T]] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[JList[T]]] JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(f.returnType)))(kcm, vcm) @@ -179,8 +177,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * mapping to that key. */ def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = { - implicit val kcm: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] implicit val vcm: ClassTag[JList[T]] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[JList[T]]] JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(f.returnType)))(kcm, vcm) 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 d5b77357a2..acf328aa6a 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 @@ -83,8 +83,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork /** Distribute a local Scala collection to form an RDD. */ def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices) } @@ -95,10 +94,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork /** Distribute a local Scala collection to form an RDD. */ def parallelizePairs[K, V](list: java.util.List[Tuple2[K, V]], numSlices: Int) : JavaPairRDD[K, V] = { - implicit val kcm: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val vcm: ClassTag[V] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val vcm: ClassTag[V] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] JavaPairRDD.fromRDD(sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices)) } @@ -154,8 +151,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * that there's very little effort required to save arbitrary objects. */ def objectFile[T](path: String, minSplits: Int): JavaRDD[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] sc.objectFile(path, minSplits)(cm) } @@ -167,8 +163,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * that there's very little effort required to save arbitrary objects. */ def objectFile[T](path: String): JavaRDD[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] sc.objectFile(path)(cm) } diff --git a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala index ed8fea97fc..bdb01f7670 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala +++ b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala @@ -23,5 +23,5 @@ import scala.reflect.ClassTag * A function that returns zero or more output records from each input record. */ abstract class FlatMapFunction[T, R] extends Function[T, java.lang.Iterable[R]] { - def elementType() : ClassTag[R] = ClassTag.Any.asInstanceOf[ClassTag[R]] + def elementType(): ClassTag[R] = ClassTag.Any.asInstanceOf[ClassTag[R]] } 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 2bf7ac256e..a659cc06c2 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 @@ -148,7 +148,7 @@ private[spark] class PythonRDD[T: ClassTag]( case eof: EOFException => { throw new SparkException("Python worker exited unexpectedly (crashed)", eof) } - case e : Throwable => throw e + case e: Throwable => throw e } } @@ -200,7 +200,7 @@ private[spark] object PythonRDD { } } catch { case eof: EOFException => {} - case e : Throwable => throw e + case e: Throwable => throw e } JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala index 72f84c99fc..37dfa7fec0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala @@ -17,8 +17,7 @@ package org.apache.spark.deploy -private[spark] object ExecutorState - extends Enumeration { +private[spark] object ExecutorState extends Enumeration { val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index d0d65ca72a..4d95efa73a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -127,7 +127,7 @@ private[spark] class Client( val fullId = appId + "/" + id logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, cores)) listener.executorAdded(fullId, workerId, hostPort, cores, memory) - + case ExecutorUpdated(id, state, message, exitStatus) => val fullId = appId + "/" + id val messageText = message.map(s => " (" + s + ")").getOrElse("") diff --git a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala index d2a3d60965..87b950ba43 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala @@ -44,7 +44,7 @@ class CartesianPartition( } private[spark] -class CartesianRDD[T: ClassTag, U:ClassTag]( +class CartesianRDD[T: ClassTag, U: ClassTag]( sc: SparkContext, var rdd1 : RDD[T], var rdd2 : RDD[U]) 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 99c34c6cc5..a84e5f9fd8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -17,9 +17,9 @@ package org.apache.spark.rdd -import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} import scala.reflect.ClassTag +import org.apache.spark.{Partition, SparkContext, TaskContext} /** * An RDD that is empty, i.e. has no element in it. diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 0c2a051a42..48168e152e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -25,7 +25,7 @@ import java.util.{HashMap => JHashMap} import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ -import scala.reflect.{ ClassTag, classTag} +import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.mapred._ import org.apache.hadoop.io.compress.CompressionCodec diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index d31a5d5177..35de13c385 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -18,9 +18,7 @@ package org.apache.spark.scheduler -private[spark] object TaskLocality - extends Enumeration -{ +private[spark] object TaskLocality extends Enumeration { // process local is expected to be used ONLY within tasksetmanager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value 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 7e721a49a5..19a025a329 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -865,7 +865,6 @@ private[spark] object BlockManager extends Logging { } def getHeartBeatFrequencyFromSystemProperties: Long = - System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 def getDisableHeartBeatsForTesting: Boolean = diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c322cbd6b5..441dcc18fb 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -84,9 +84,9 @@ object SparkBuild extends Build { case Some(v) => v.toBoolean } - if(isNewHadoop && isYarnEnabled) { - println("""Yarn with Hadoop version 2.2.x is not yet expected to work. - Please set env SPARK_HADOOP_VERSION to appropriate version or set SPARK_YARN to false.""") + if (isNewHadoop && isYarnEnabled) { + println( """Yarn with Hadoop version 2.2.x is not yet expected to work. + Please set env SPARK_HADOOP_VERSION to appropriate version or set SPARK_YARN to false.""") throw new Exception("Yarn with Hadoop version 2.2.x is not yet expected to work.") } diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala index 85b0978c81..b2e1df173e 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala @@ -12,7 +12,6 @@ import scala.reflect.internal.util.BatchSourceFile import scala.tools.nsc.ast.parser.Tokens.EOF import org.apache.spark.Logging -import scala.Some trait SparkExprTyper extends Logging { val repl: SparkIMain -- cgit v1.2.3 From 1ae3c0fc5e0273319d0e9497268a4c9854aae2e0 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sat, 14 Dec 2013 10:44:24 +0530 Subject: Added a comment about ActorRef and ActorSelection difference. --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 7 +++++++ 1 file changed, 7 insertions(+) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index fbda11f578..10fae5af9f 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -72,6 +72,13 @@ private[spark] class MapOutputTracker extends Logging { // throw a SparkException if this fails. private def askTracker(message: Any): Any = { try { + /* + The difference between ActorRef and ActorSelection is well explained here: + http://doc.akka.io/docs/akka/2.2.3/project/migration-guide-2.1.x-2.2.x.html#Use_actorSelection_instead_of_actorFor + In spark a map output tracker can be either started on Driver where it is created which + is an ActorRef or it can be on executor from where it is looked up which is an + actorSelection. + */ val future = trackerActor match { case Left(a: ActorRef) => a.ask(message)(timeout) case Right(b: ActorSelection) => b.ask(message)(timeout) -- cgit v1.2.3 From 8a397a959bf0b68f7d10fa57665225e0c2b5d03a Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Mon, 16 Dec 2013 12:07:39 -0500 Subject: Track task value serialisation time in TaskMetrics. --- .../scala/org/apache/spark/executor/Executor.scala | 12 ++++++++---- .../org/apache/spark/executor/TaskMetrics.scala | 5 +++++ .../org/apache/spark/scheduler/TaskResult.scala | 20 ++++++++++---------- .../cluster/ClusterTaskSetManagerSuite.scala | 4 +++- 4 files changed, 26 insertions(+), 15 deletions(-) (limited to 'core/src') 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 0b0a60ee60..02ad64d070 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -222,18 +222,22 @@ private[spark] class Executor( return } + val objectSer = SparkEnv.get.serializer.newInstance() + val beforeSerialization = System.currentTimeMillis() + val valueBytes = objectSer.serialize(value) + val afterSerialization = System.currentTimeMillis() + for (m <- task.metrics) { m.hostname = Utils.localHostName() m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt m.jvmGCTime = gcTime - startGCTime + m.serializationTime = (afterSerialization - beforeSerialization).toInt } - // 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 just change the relevants bytes in the byte buffer + val accumUpdates = Accumulators.values - val directResult = new DirectTaskResult(value, accumUpdates, task.metrics.getOrElse(null)) + val directResult = new DirectTaskResult(valueBytes, accumUpdates, task.metrics.getOrElse(null)) val serializedDirectResult = ser.serialize(directResult) logInfo("Serialized size of result for " + taskId + " is " + serializedDirectResult.limit) val serializedResult = { diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index c0ce46e379..c036866afd 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -43,6 +43,11 @@ class TaskMetrics extends Serializable { */ var jvmGCTime: Long = _ + /** + * Amount of time spent serializing the result of the task + */ + var serializationTime: Long = _ + /** * If this task reads from shuffle output, metrics on getting shuffle data will be collected here */ 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 7e468d0d67..4e00bc8271 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -35,18 +35,15 @@ case class IndirectTaskResult[T](blockId: BlockId) extends TaskResult[T] with Se /** A TaskResult that contains the task's return value and accumulator updates. */ private[spark] -class DirectTaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics) +class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics) extends TaskResult[T] with Externalizable { - def this() = this(null.asInstanceOf[T], null, null) + def this() = this(null.asInstanceOf[ByteBuffer], null, null) override def writeExternal(out: ObjectOutput) { - val objectSer = SparkEnv.get.serializer.newInstance() - val bb = objectSer.serialize(value) - - out.writeInt(bb.remaining()) - Utils.writeByteBuffer(bb, out) + out.writeInt(valueBytes.remaining); + Utils.writeByteBuffer(valueBytes, out) out.writeInt(accumUpdates.size) for ((key, value) <- accumUpdates) { @@ -58,12 +55,10 @@ class DirectTaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var me override def readExternal(in: ObjectInput) { - 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)) + valueBytes = ByteBuffer.wrap(byteVal) val numUpdates = in.readInt if (numUpdates == 0) { @@ -76,4 +71,9 @@ class DirectTaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var me } metrics = in.readObject().asInstanceOf[TaskMetrics] } + + def value(): T = { + val objectSer = SparkEnv.get.serializer.newInstance() + return objectSer.deserialize(valueBytes) + } } 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 b97f2b19b5..788cbb81bf 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 @@ -313,6 +313,8 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo } def createTaskResult(id: Int): DirectTaskResult[Int] = { - new DirectTaskResult[Int](id, mutable.Map.empty, new TaskMetrics) + val objectSer = SparkEnv.get.serializer.newInstance() + new DirectTaskResult[Int](objectSer.serialize(id), mutable.Map.empty, + new TaskMetrics) } } -- cgit v1.2.3 From 882d544856c61573cdd6124e921f700d580d170d Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Mon, 16 Dec 2013 13:27:03 -0500 Subject: UI to display serialisation time of a stage. --- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 6 ++++++ 1 file changed, 6 insertions(+) (limited to 'core/src') 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 69f9446bab..81651bdd20 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 @@ -101,6 +101,11 @@ private[spark] class StagePage(parent: JobProgressUI) { None } else { + val serializationTimes = validTasks.map{case (info, metrics, exception) => + metrics.get.serializationTime.toDouble} + val serializationQuantiles = "Serialization Time" +: Distribution(serializationTimes).get.getQuantiles().map( + ms => parent.formatDuration(ms.toLong)) + val serviceTimes = validTasks.map{case (info, metrics, exception) => metrics.get.executorRunTime.toDouble} val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map( @@ -149,6 +154,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes) val listings: Seq[Seq[String]] = Seq( + serializationQuantiles, serviceQuantiles, gettingResultQuantiles, schedulerDelayQuantiles, -- cgit v1.2.3 From 963d6f065a763c2b94529bbd3ac4326e190bb2d7 Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Mon, 16 Dec 2013 23:14:52 -0500 Subject: Incorporate pwendell's code review suggestions. --- core/src/main/scala/org/apache/spark/executor/Executor.scala | 6 +++--- core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala | 4 ++-- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 4 ++-- .../apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala | 3 +-- 4 files changed, 8 insertions(+), 9 deletions(-) (limited to 'core/src') 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 02ad64d070..0f19d7a96b 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -222,9 +222,9 @@ private[spark] class Executor( return } - val objectSer = SparkEnv.get.serializer.newInstance() + val resultSer = SparkEnv.get.serializer.newInstance() val beforeSerialization = System.currentTimeMillis() - val valueBytes = objectSer.serialize(value) + val valueBytes = resultSer.serialize(value) val afterSerialization = System.currentTimeMillis() for (m <- task.metrics) { @@ -232,7 +232,7 @@ private[spark] class Executor( m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt m.jvmGCTime = gcTime - startGCTime - m.serializationTime = (afterSerialization - beforeSerialization).toInt + m.resultSerializationTime = (afterSerialization - beforeSerialization).toInt } val accumUpdates = Accumulators.values diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index c036866afd..bb1471d9ee 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -44,9 +44,9 @@ class TaskMetrics extends Serializable { var jvmGCTime: Long = _ /** - * Amount of time spent serializing the result of the task + * Amount of time spent serializing the task result */ - var serializationTime: Long = _ + var resultSerializationTime: Long = _ /** * If this task reads from shuffle output, metrics on getting shuffle data will be collected here 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 81651bdd20..2f06efa66e 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 @@ -102,8 +102,8 @@ private[spark] class StagePage(parent: JobProgressUI) { } else { val serializationTimes = validTasks.map{case (info, metrics, exception) => - metrics.get.serializationTime.toDouble} - val serializationQuantiles = "Serialization Time" +: Distribution(serializationTimes).get.getQuantiles().map( + metrics.get.resultSerializationTime.toDouble} + val serializationQuantiles = "Result serialization time" +: Distribution(serializationTimes).get.getQuantiles().map( ms => parent.formatDuration(ms.toLong)) val serviceTimes = validTasks.map{case (info, metrics, exception) => 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 788cbb81bf..2476ab5c19 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 @@ -314,7 +314,6 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo def createTaskResult(id: Int): DirectTaskResult[Int] = { val objectSer = SparkEnv.get.serializer.newInstance() - new DirectTaskResult[Int](objectSer.serialize(id), mutable.Map.empty, - new TaskMetrics) + new DirectTaskResult[Int](objectSer.serialize(id), mutable.Map.empty, new TaskMetrics) } } -- cgit v1.2.3 From b2f0329511f3caaf473cf300792690703a300a22 Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Tue, 17 Dec 2013 00:18:46 -0500 Subject: Missed a spot; had an objectSer here too. --- core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') 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 4e00bc8271..e80cc6b0f6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -73,7 +73,7 @@ class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long } def value(): T = { - val objectSer = SparkEnv.get.serializer.newInstance() - return objectSer.deserialize(valueBytes) + val resultSer = SparkEnv.get.serializer.newInstance() + return resultSer.deserialize(valueBytes) } } -- cgit v1.2.3 From 36060f4f50ead2632117bb12e8c5bc1fb4f91f1e Mon Sep 17 00:00:00 2001 From: "wangda.tan" Date: Tue, 17 Dec 2013 17:55:38 +0800 Subject: spark-898, changes according to review comments --- .../org/apache/spark/ui/exec/ExecutorsUI.scala | 39 +++++++++++++++++-- .../org/apache/spark/ui/jobs/ExecutorSummary.scala | 3 +- .../org/apache/spark/ui/jobs/ExecutorTable.scala | 40 ++++++++++--------- .../scala/org/apache/spark/ui/jobs/IndexPage.scala | 5 +-- .../apache/spark/ui/jobs/JobProgressListener.scala | 31 ++++++++------- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 3 +- .../spark/ui/jobs/JobProgressListenerSuite.scala | 45 ++++++---------------- 7 files changed, 90 insertions(+), 76 deletions(-) (limited to 'core/src') 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 e596690bc3..808bbe8c8f 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 @@ -56,7 +56,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { 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") + "Active tasks", "Failed tasks", "Complete tasks", "Total tasks", "Duration", "Shuffle Read", + "Shuffle Write") def execRow(kv: Seq[String]) = { @@ -73,6 +74,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { {kv(7)} {kv(8)} {kv(9)} + {Utils.msDurationToString(kv(10).toLong)} + {Utils.bytesToString(kv(11).toLong)} + {Utils.bytesToString(kv(12).toLong)} } @@ -111,6 +115,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0) val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0) val totalTasks = activeTasks + failedTasks + completedTasks + val totalDuration = listener.executorToDuration.getOrElse(execId, 0) + val totalShuffleRead = listener.executorToShuffleRead.getOrElse(execId, 0) + val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0) Seq( execId, @@ -122,7 +129,10 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { activeTasks.toString, failedTasks.toString, completedTasks.toString, - totalTasks.toString + totalTasks.toString, + totalDuration.toString, + totalShuffleRead.toString, + totalShuffleWrite.toString ) } @@ -130,6 +140,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val executorToTasksActive = HashMap[String, HashSet[TaskInfo]]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() + val executorToDuration = HashMap[String, Long]() + val executorToShuffleRead = HashMap[String, Long]() + val executorToShuffleWrite = HashMap[String, Long]() override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId @@ -137,9 +150,12 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { activeTasks += taskStart.taskInfo } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { val eid = taskEnd.taskInfo.executorId val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) + val newDuration = executorToDuration.getOrElse(eid, 0L) + taskEnd.taskInfo.duration + executorToDuration.put(eid, newDuration) + activeTasks -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { @@ -150,6 +166,23 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 (None, Option(taskEnd.taskMetrics)) } + + // update shuffle read/write + val shuffleRead = taskEnd.taskMetrics.shuffleReadMetrics + shuffleRead match { + case Some(s) => + val newShuffleRead = executorToShuffleRead.getOrElse(eid, 0L) + s.remoteBytesRead + executorToShuffleRead.put(eid, newShuffleRead) + case _ => {} + } + val shuffleWrite = taskEnd.taskMetrics.shuffleWriteMetrics + shuffleWrite match { + case Some(s) => { + val newShuffleWrite = executorToShuffleWrite.getOrElse(eid, 0L) + s.shuffleBytesWritten + executorToShuffleWrite.put(eid, newShuffleWrite) + } + case _ => {} + } } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala index f2ee12081c..75c0dd2c7f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala @@ -19,9 +19,8 @@ package org.apache.spark.ui.jobs private[spark] class ExecutorSummary() { var duration : Long = 0 - var totalTasks : Int = 0 var failedTasks : Int = 0 - var succeedTasks : Int = 0 + var succeededTasks : Int = 0 var shuffleRead : Long = 0 var shuffleWrite : Long = 0 } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index c6823cd823..763d5a344b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -17,14 +17,13 @@ package org.apache.spark.ui.jobs - import scala.xml.Node import org.apache.spark.scheduler.SchedulingMode - +import org.apache.spark.util.Utils /** Page showing executor summary */ -private[spark] class ExecutorTable(val parent: JobProgressUI) { +private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) { val listener = parent.listener val dateFmt = parent.dateFmt @@ -42,9 +41,9 @@ private[spark] class ExecutorTable(val parent: JobProgressUI) { Executor ID Duration - #Tasks - #Failed Tasks - #Succeed Tasks + Total Tasks + Failed Tasks + Succeeded Tasks Shuffle Read Shuffle Write @@ -55,19 +54,24 @@ private[spark] class ExecutorTable(val parent: JobProgressUI) { } private def createExecutorTable() : Seq[Node] = { - val executorIdToSummary = listener.executorIdToSummary - executorIdToSummary.toSeq.sortBy(_._1).map{ - case (k,v) => { - - {k} - {v.duration} ms - {v.totalTasks} - {v.failedTasks} - {v.succeedTasks} - {v.shuffleRead} - {v.shuffleWrite} - + val executorIdToSummary = listener.stageIdToExecutorSummaries.get(stageId) + executorIdToSummary match { + case Some(x) => { + x.toSeq.sortBy(_._1).map{ + case (k,v) => { + + {k} + {parent.formatDuration(v.duration)} + {v.failedTasks + v.succeededTasks} + {v.failedTasks} + {v.succeededTasks} + {Utils.bytesToString(v.shuffleRead)} + {Utils.bytesToString(v.shuffleWrite)} + + } + } } + case _ => { Seq[Node]() } } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 653a84b60f..854afb665a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -45,7 +45,6 @@ private[spark] class IndexPage(parent: JobProgressUI) { val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) val completedStagesTable = new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) - val executorTable = new ExecutorTable(parent) val pools = listener.sc.getAllPools val poolTable = new PoolTable(pools, listener) @@ -59,7 +58,7 @@ private[spark] class IndexPage(parent: JobProgressUI) {
  • Scheduling Mode: {parent.sc.getSchedulingMode}
  • Executor Summary: - {listener.executorIdToSummary.size} + {listener.stageIdToExecutorSummaries.size}
  • Active Stages: @@ -82,8 +81,6 @@ private[spark] class IndexPage(parent: JobProgressUI) { } else { Seq() }} ++ -

    Executor Summary

    ++ - executorTable.toNodeSeq++

    Active Stages ({activeStages.size})

    ++ activeStagesTable.toNodeSeq++

    Completed Stages ({completedStages.size})

    ++ 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 2635478592..8c92ff19a6 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 @@ -57,7 +57,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList val stageIdToTasksFailed = HashMap[Int, Int]() val stageIdToTaskInfos = HashMap[Int, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() - val executorIdToSummary = HashMap[String, ExecutorSummary]() + val stageIdToExecutorSummaries = HashMap[Int, HashMap[String, ExecutorSummary]]() override def onJobStart(jobStart: SparkListenerJobStart) {} @@ -115,9 +115,6 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList sid, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) stageIdToTaskInfos(sid) = taskList - val executorSummary = executorIdToSummary.getOrElseUpdate(key = taskStart.taskInfo.executorId, - op = new ExecutorSummary()) - executorSummary.totalTasks += 1 } override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) @@ -127,32 +124,39 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { - // update executor summary - val executorSummary = executorIdToSummary.get(taskEnd.taskInfo.executorId) + val sid = taskEnd.task.stageId + + // create executor summary map if necessary + val executorSummaryMap = stageIdToExecutorSummaries.getOrElseUpdate(key = sid, + op = new HashMap[String, ExecutorSummary]()) + executorSummaryMap.getOrElseUpdate(key = taskEnd.taskInfo.executorId, + op = new ExecutorSummary()) + + val executorSummary = executorSummaryMap.get(taskEnd.taskInfo.executorId) executorSummary match { - case Some(x) => { + case Some(y) => { // first update failed-task, succeed-task taskEnd.reason match { - case e: ExceptionFailure => - x.failedTasks += 1 + case Success => + y.succeededTasks += 1 case _ => - x.succeedTasks += 1 + y.failedTasks += 1 } // update duration - x.duration += taskEnd.taskInfo.duration + y.duration += taskEnd.taskInfo.duration // update shuffle read/write val shuffleRead = taskEnd.taskMetrics.shuffleReadMetrics shuffleRead match { case Some(s) => - x.shuffleRead += s.remoteBytesRead + y.shuffleRead += s.remoteBytesRead case _ => {} } val shuffleWrite = taskEnd.taskMetrics.shuffleWriteMetrics shuffleWrite match { case Some(s) => { - x.shuffleWrite += s.shuffleBytesWritten + y.shuffleWrite += s.shuffleBytesWritten } case _ => {} } @@ -160,7 +164,6 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList case _ => {} } - val sid = taskEnd.task.stageId val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) tasksActive -= taskEnd.taskInfo 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 69f9446bab..c077613b1d 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 @@ -160,9 +160,10 @@ private[spark] class StagePage(parent: JobProgressUI) { def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} Some(listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true)) } - + val executorTable = new ExecutorTable(parent, stageId) val content = summary ++ +

    Summary Metrics for Executors

    ++ executorTable.toNodeSeq() ++

    Summary Metrics for {numCompleted} Completed Tasks

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

    Tasks

    ++ taskTable diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 861d37a862..67a57a0e7f 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -19,26 +19,19 @@ package org.apache.spark.ui.jobs import org.scalatest.FunSuite import org.apache.spark.scheduler._ -import org.apache.spark.SparkContext -import org.apache.spark.Success +import org.apache.spark.{LocalSparkContext, SparkContext, Success} import org.apache.spark.scheduler.SparkListenerTaskStart import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} -class JobProgressListenerSuite extends FunSuite { +class JobProgressListenerSuite extends FunSuite with LocalSparkContext { test("test executor id to summary") { - val sc = new SparkContext("local", "joblogger") + val sc = new SparkContext("local", "test") val listener = new JobProgressListener(sc) val taskMetrics = new TaskMetrics() val shuffleReadMetrics = new ShuffleReadMetrics() // nothing in it - assert(listener.executorIdToSummary.size == 0) - - // launched a task, should get an item in map - listener.onTaskStart(new SparkListenerTaskStart( - new ShuffleMapTask(0, null, null, 0, null), - new TaskInfo(1234L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL))) - assert(listener.executorIdToSummary.size == 1) + assert(listener.stageIdToExecutorSummaries.size == 0) // finish this task, should get updated shuffleRead shuffleReadMetrics.remoteBytesRead = 1000 @@ -47,20 +40,15 @@ class JobProgressListenerSuite extends FunSuite { taskInfo.finishTime = 1 listener.onTaskEnd(new SparkListenerTaskEnd( new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) - assert(listener.executorIdToSummary.getOrElse("exe-1", fail()).shuffleRead == 1000) + assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-1", fail()) + .shuffleRead == 1000) // finish a task with unknown executor-id, nothing should happen taskInfo = new TaskInfo(1234L, 0, 1000L, "exe-unknown", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 listener.onTaskEnd(new SparkListenerTaskEnd( new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) - assert(listener.executorIdToSummary.size == 1) - - // launched a task - listener.onTaskStart(new SparkListenerTaskStart( - new ShuffleMapTask(0, null, null, 0, null), - new TaskInfo(1235L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL))) - assert(listener.executorIdToSummary.size == 1) + assert(listener.stageIdToExecutorSummaries.size == 1) // finish this task, should get updated duration shuffleReadMetrics.remoteBytesRead = 1000 @@ -69,13 +57,8 @@ class JobProgressListenerSuite extends FunSuite { taskInfo.finishTime = 1 listener.onTaskEnd(new SparkListenerTaskEnd( new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) - assert(listener.executorIdToSummary.getOrElse("exe-1", fail()).shuffleRead == 2000) - - // launched a task in another exec - listener.onTaskStart(new SparkListenerTaskStart( - new ShuffleMapTask(0, null, null, 0, null), - new TaskInfo(1236L, 0, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL))) - assert(listener.executorIdToSummary.size == 2) + assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-1", fail()) + .shuffleRead == 2000) // finish this task, should get updated duration shuffleReadMetrics.remoteBytesRead = 1000 @@ -84,13 +67,7 @@ class JobProgressListenerSuite extends FunSuite { taskInfo.finishTime = 1 listener.onTaskEnd(new SparkListenerTaskEnd( new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) - assert(listener.executorIdToSummary.getOrElse("exe-2", fail()).shuffleRead == 1000) - - // do finalize - sc.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") + assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-2", fail()) + .shuffleRead == 1000) } } -- cgit v1.2.3 From 59e53fa21caa202a57093c74ada128fca2be5bac Mon Sep 17 00:00:00 2001 From: "wangda.tan" Date: Tue, 17 Dec 2013 17:57:27 +0800 Subject: spark-968, changes for avoid a NPE --- .../org/apache/spark/ui/exec/ExecutorsUI.scala | 30 ++++++++++++---------- .../apache/spark/ui/jobs/JobProgressListener.scala | 24 +++++++++-------- 2 files changed, 29 insertions(+), 25 deletions(-) (limited to 'core/src') 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 808bbe8c8f..f62ae37466 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 @@ -150,7 +150,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { activeTasks += taskStart.taskInfo } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) val newDuration = executorToDuration.getOrElse(eid, 0L) + taskEnd.taskInfo.duration @@ -168,20 +168,22 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } // update shuffle read/write - val shuffleRead = taskEnd.taskMetrics.shuffleReadMetrics - shuffleRead match { - case Some(s) => - val newShuffleRead = executorToShuffleRead.getOrElse(eid, 0L) + s.remoteBytesRead - executorToShuffleRead.put(eid, newShuffleRead) - case _ => {} - } - val shuffleWrite = taskEnd.taskMetrics.shuffleWriteMetrics - shuffleWrite match { - case Some(s) => { - val newShuffleWrite = executorToShuffleWrite.getOrElse(eid, 0L) + s.shuffleBytesWritten - executorToShuffleWrite.put(eid, newShuffleWrite) + if (null != taskEnd.taskMetrics) { + val shuffleRead = taskEnd.taskMetrics.shuffleReadMetrics + shuffleRead match { + case Some(s) => + val newShuffleRead = executorToShuffleRead.getOrElse(eid, 0L) + s.remoteBytesRead + executorToShuffleRead.put(eid, newShuffleRead) + case _ => {} + } + val shuffleWrite = taskEnd.taskMetrics.shuffleWriteMetrics + shuffleWrite match { + case Some(s) => { + val newShuffleWrite = executorToShuffleWrite.getOrElse(eid, 0L) + s.shuffleBytesWritten + executorToShuffleWrite.put(eid, newShuffleWrite) + } + case _ => {} } - case _ => {} } } } 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 8c92ff19a6..64ce715993 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 @@ -147,18 +147,20 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList y.duration += taskEnd.taskInfo.duration // update shuffle read/write - val shuffleRead = taskEnd.taskMetrics.shuffleReadMetrics - shuffleRead match { - case Some(s) => - y.shuffleRead += s.remoteBytesRead - case _ => {} - } - val shuffleWrite = taskEnd.taskMetrics.shuffleWriteMetrics - shuffleWrite match { - case Some(s) => { - y.shuffleWrite += s.shuffleBytesWritten + if (null != taskEnd.taskMetrics) { + val shuffleRead = taskEnd.taskMetrics.shuffleReadMetrics + shuffleRead match { + case Some(s) => + y.shuffleRead += s.remoteBytesRead + case _ => {} + } + val shuffleWrite = taskEnd.taskMetrics.shuffleWriteMetrics + shuffleWrite match { + case Some(s) => { + y.shuffleWrite += s.shuffleBytesWritten + } + case _ => {} } - case _ => {} } } case _ => {} -- cgit v1.2.3 From 9a6864d016f0f923c885ca2a4977104cd19ded4f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 17 Dec 2013 18:44:39 -0800 Subject: Fixed a performance problem in RDD.top and BoundedPriorityQueue (size in BoundedPriority was actually traversing the entire queue to calculate the size, resulting in bad performance in insertion). --- core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala | 2 ++ 1 file changed, 2 insertions(+) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index 0b51c23f7b..a38329df03 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -34,6 +34,8 @@ class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) override def iterator: Iterator[A] = underlying.iterator.asScala + override def size: Int = underlying.size + override def ++=(xs: TraversableOnce[A]): this.type = { xs.foreach { this += _ } this -- cgit v1.2.3 From 717c7fddb27a3ec8732a760c000bbfa7060d76c1 Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Tue, 17 Dec 2013 23:02:21 -0500 Subject: objectSer -> valueSer in a test. --- .../apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') 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 cb719d7ab9..bb28a31a99 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 @@ -313,7 +313,7 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo } def createTaskResult(id: Int): DirectTaskResult[Int] = { - val objectSer = SparkEnv.get.serializer.newInstance() - new DirectTaskResult[Int](objectSer.serialize(id), mutable.Map.empty, new TaskMetrics) + val valueSer = SparkEnv.get.serializer.newInstance() + new DirectTaskResult[Int](valueSer.serialize(id), mutable.Map.empty, new TaskMetrics) } } -- cgit v1.2.3 From d3b1af4b6c7766bbf7a09ee6d5c1b13eda6b098f Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Wed, 18 Dec 2013 14:25:56 -0500 Subject: Add a serialisation time column to the StagePage. --- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) (limited to 'core/src') 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 2f06efa66e..996e1b4d1a 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 @@ -86,7 +86,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val taskHeaders: Seq[String] = Seq("Task Index", "Task ID", "Status", "Locality Level", "Executor", "Launch Time") ++ - Seq("Duration", "GC Time") ++ + Seq("Duration", "GC Time", "Result Ser Time") ++ {if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++ {if (hasShuffleWrite) Seq("Write Time", "Shuffle Write") else Nil} ++ Seq("Errors") @@ -189,6 +189,7 @@ private[spark] class StagePage(parent: JobProgressUI) { 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) + val serializationTime = metrics.map(m => m.resultSerializationTime).getOrElse(0L) val maybeShuffleRead = metrics.flatMap{m => m.shuffleReadMetrics}.map{s => s.remoteBytesRead} val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") @@ -216,6 +217,9 @@ private[spark] class StagePage(parent: JobProgressUI) { {if (gcTime > 0) parent.formatDuration(gcTime) else ""} + + {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} + {if (shuffleRead) { {shuffleReadReadable} -- cgit v1.2.3 From af0cd6bd27dda73b326bcb6a66addceadebf5e54 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 18 Dec 2013 11:40:07 -0800 Subject: Add collectPartition to JavaRDD interface. Also remove takePartition from PythonRDD and use collectPartition in rdd.py. --- .../org/apache/spark/api/java/JavaRDDLike.scala | 11 ++++++++- .../org/apache/spark/api/python/PythonRDD.scala | 4 ---- .../test/scala/org/apache/spark/JavaAPISuite.java | 28 ++++++++++++++++++++++ python/pyspark/context.py | 3 --- python/pyspark/rdd.py | 2 +- 5 files changed, 39 insertions(+), 9 deletions(-) (limited to 'core/src') 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 9e912d3adb..1d71875ed1 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 @@ -26,7 +26,7 @@ import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec import org.apache.spark.{SparkContext, Partition, TaskContext} -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RDD, PartitionPruningRDD} 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} @@ -244,6 +244,15 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { new java.util.ArrayList(arr) } + /** + * Return an array that contains all of the elements in a specific partition of this RDD. + */ + def collectPartition(partitionId: Int): JList[T] = { + import scala.collection.JavaConversions._ + val partition = new PartitionPruningRDD[T](rdd, _ == partitionId) + new java.util.ArrayList(partition.collect().toSeq) + } + /** * Reduces the elements of this RDD using the specified commutative and associative binary operator. */ 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 a659cc06c2..ca42c76928 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 @@ -235,10 +235,6 @@ private[spark] object PythonRDD { file.close() } - def takePartition[T](rdd: RDD[T], partition: Int): Iterator[T] = { - implicit val cm : ClassTag[T] = rdd.elementClassTag - rdd.context.runJob(rdd, ((x: Iterator[T]) => x.toArray), Seq(partition), true).head.iterator - } } private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] { diff --git a/core/src/test/scala/org/apache/spark/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java index 4234f6eac7..2862ed3019 100644 --- a/core/src/test/scala/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java @@ -897,4 +897,32 @@ public class JavaAPISuite implements Serializable { new Tuple2(0, 4)), rdd3.collect()); } + + @Test + public void collectPartition() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); + + JavaPairRDD rdd2 = rdd1.map(new PairFunction() { + @Override + public Tuple2 call(Integer i) throws Exception { + return new Tuple2(i, i % 2); + } + }); + + Assert.assertEquals(Arrays.asList(1, 2), rdd1.collectPartition(0)); + Assert.assertEquals(Arrays.asList(3, 4), rdd1.collectPartition(1)); + Assert.assertEquals(Arrays.asList(5, 6, 7), rdd1.collectPartition(2)); + + Assert.assertEquals(Arrays.asList(new Tuple2(1, 1), + new Tuple2(2, 0)), + rdd2.collectPartition(0)); + Assert.assertEquals(Arrays.asList(new Tuple2(3, 1), + new Tuple2(4, 0)), + rdd2.collectPartition(1)); + Assert.assertEquals(Arrays.asList(new Tuple2(5, 1), + new Tuple2(6, 0), + new Tuple2(7, 1)), + rdd2.collectPartition(2)); + } + } diff --git a/python/pyspark/context.py b/python/pyspark/context.py index cbd41e58c4..0604f6836c 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -43,7 +43,6 @@ class SparkContext(object): _gateway = None _jvm = None _writeToFile = None - _takePartition = None _next_accum_id = 0 _active_spark_context = None _lock = Lock() @@ -134,8 +133,6 @@ class SparkContext(object): SparkContext._jvm = SparkContext._gateway.jvm SparkContext._writeToFile = \ SparkContext._jvm.PythonRDD.writeToFile - SparkContext._takePartition = \ - SparkContext._jvm.PythonRDD.takePartition if instance: if SparkContext._active_spark_context and SparkContext._active_spark_context != instance: diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 61720dcf1a..d81b7c90c1 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -577,7 +577,7 @@ class RDD(object): mapped = self.mapPartitions(takeUpToNum) items = [] for partition in range(mapped._jrdd.splits().size()): - iterator = self.ctx._takePartition(mapped._jrdd.rdd(), partition) + iterator = mapped._jrdd.collectPartition(partition).iterator() items.extend(mapped._collect_iterator_through_file(iterator)) if len(items) >= num: break -- cgit v1.2.3 From b80ec05635132f96772545803a10a1bbfa1250e7 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 18 Dec 2013 15:35:24 -0800 Subject: Added StatsReportListener to generate processing time statistics across multiple batches. --- .../org/apache/spark/scheduler/SparkListener.scala | 5 +-- .../spark/streaming/scheduler/JobScheduler.scala | 2 +- .../streaming/scheduler/StreamingListener.scala | 45 +++++++++++++++++++++- 3 files changed, 46 insertions(+), 6 deletions(-) (limited to 'core/src') 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 2c5d87419d..ee63b3c4a1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -131,8 +131,8 @@ object StatsReportListener extends Logging { def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) { val stats = d.statCounter - logInfo(heading + stats) val quantiles = d.getQuantiles(probabilities).map{formatNumber} + logInfo(heading + stats) logInfo(percentilesHeader) logInfo("\t" + quantiles.mkString("\t")) } @@ -173,8 +173,6 @@ object StatsReportListener extends Logging { showMillisDistribution(heading, extractLongDistribution(stage, getMetric)) } - - val seconds = 1000L val minutes = seconds * 60 val hours = minutes * 60 @@ -198,7 +196,6 @@ object StatsReportListener extends Logging { } - case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double) object RuntimePercentage { def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 14906fd720..69930f3b6c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -79,13 +79,13 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { jobSet.afterJobStop(job) logInfo("Finished job " + job.id + " from job set of time " + jobSet.time) if (jobSet.hasCompleted) { - listenerBus.post(StreamingListenerBatchCompleted(jobSet.toBatchInfo())) jobSets.remove(jobSet.time) generator.onBatchCompletion(jobSet.time) logInfo("Total delay: %.3f s for time %s (execution: %.3f s)".format( jobSet.totalDelay / 1000.0, jobSet.time.toString, jobSet.processingDelay / 1000.0 )) + listenerBus.post(StreamingListenerBatchCompleted(jobSet.toBatchInfo())) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala index 49fd0d29c3..5647ffab8d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala @@ -17,14 +17,22 @@ package org.apache.spark.streaming.scheduler +import scala.collection.mutable.Queue +import org.apache.spark.util.Distribution + +/** Base trait for events related to StreamingListener */ sealed trait StreamingListenerEvent case class StreamingListenerBatchCompleted(batchInfo: BatchInfo) extends StreamingListenerEvent case class StreamingListenerBatchStarted(batchInfo: BatchInfo) extends StreamingListenerEvent -trait StreamingListener { +/** + * A listener interface for receiving information about an ongoing streaming + * computation. + */ +trait StreamingListener { /** * Called when processing of a batch has completed */ @@ -34,4 +42,39 @@ trait StreamingListener { * Called when processing of a batch has started */ def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { } +} + + +/** + * A simple StreamingListener that logs summary statistics across Spark Streaming batches + * @param numBatchInfos Number of last batches to consider for generating statistics (default: 10) + */ +class StatsReportListener(numBatchInfos: Int = 10) extends StreamingListener { + + import org.apache.spark + + val batchInfos = new Queue[BatchInfo]() + + override def onBatchCompleted(batchStarted: StreamingListenerBatchCompleted) { + addToQueue(batchStarted.batchInfo) + printStats() + } + + def addToQueue(newPoint: BatchInfo) { + batchInfos.enqueue(newPoint) + if (batchInfos.size > numBatchInfos) batchInfos.dequeue() + } + + def printStats() { + showMillisDistribution("Total delay: ", _.totalDelay) + showMillisDistribution("Processing time: ", _.processingDelay) + } + + def showMillisDistribution(heading: String, getMetric: BatchInfo => Option[Long]) { + spark.scheduler.StatsReportListener.showMillisDistribution(heading, extractDistribution(getMetric)) + } + + def extractDistribution(getMetric: BatchInfo => Option[Long]): Option[Distribution] = { + Distribution(batchInfos.flatMap(getMetric(_)).map(_.toDouble)) + } } \ No newline at end of file -- cgit v1.2.3 From 293a0af5a1def95e47d9188f42957083f5adf3b8 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Wed, 18 Dec 2013 18:51:02 -0800 Subject: In experimental clusters we've observed that a 10 second timeout was insufficient, despite having a low number of nodes and relatively small workload (16 nodes, <1.5 TB data). This would cause an entire job to fail at the beginning of the reduce phase. There is no particular reason for this value to be small as a timeout should only occur in an exceptional situation. Also centralized the reading of spark.akka.askTimeout to AkkaUtils (surely this can later be cleaned up to use Typesafe). Finally, deleted some lurking implicits. If anyone can think of a reason they should still be there, please let me know. --- .../main/scala/org/apache/spark/MapOutputTracker.scala | 5 ++--- .../scala/org/apache/spark/deploy/client/Client.scala | 6 +++--- .../scala/org/apache/spark/deploy/master/Master.scala | 17 ++++++----------- .../apache/spark/deploy/master/ui/ApplicationPage.scala | 14 +++++--------- .../org/apache/spark/deploy/master/ui/IndexPage.scala | 16 ++++++---------- .../org/apache/spark/deploy/master/ui/MasterWebUI.scala | 10 +++------- .../org/apache/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- .../org/apache/spark/deploy/worker/ui/WorkerWebUI.scala | 10 +++------- .../cluster/CoarseGrainedSchedulerBackend.scala | 10 ++++------ .../org/apache/spark/storage/BlockManagerMaster.scala | 7 ++++--- .../apache/spark/storage/BlockManagerMasterActor.scala | 11 ++++------- .../org/apache/spark/ui/storage/BlockManagerUI.scala | 3 --- .../main/scala/org/apache/spark/util/AkkaUtils.scala | 6 ++++++ 13 files changed, 48 insertions(+), 71 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 10fae5af9f..ccffcc356c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -29,8 +29,7 @@ import akka.pattern.ask import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{MetadataCleanerType, Utils, MetadataCleaner, TimeStampedHashMap} - +import org.apache.spark.util.{AkkaUtils, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} private[spark] sealed trait MapOutputTrackerMessage private[spark] case class GetMapOutputStatuses(shuffleId: Int, requester: String) @@ -53,7 +52,7 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster private[spark] class MapOutputTracker extends Logging { - private val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + private val timeout = AkkaUtils.askTimeout // Set to the MapOutputTrackerActor living on the driver var trackerActor: Either[ActorRef, ActorSelection] = _ diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 4d95efa73a..953755e40d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -23,14 +23,14 @@ import scala.concurrent.duration._ import scala.concurrent.Await import akka.actor._ -import akka.pattern.AskTimeoutException import akka.pattern.ask -import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent, AssociationErrorEvent} +import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} import org.apache.spark.{SparkException, Logging} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master +import org.apache.spark.util.AkkaUtils /** @@ -178,7 +178,7 @@ private[spark] class Client( def stop() { if (actor != null) { try { - val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + val timeout = AkkaUtils.askTimeout val future = actor.ask(StopClient)(timeout) Await.result(future, timeout) } catch { 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 c627dd3806..7b2b1c3327 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 @@ -18,19 +18,16 @@ package org.apache.spark.deploy.master import java.text.SimpleDateFormat -import java.util.concurrent.TimeUnit import java.util.Date import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ -import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor._ import akka.pattern.ask -import akka.remote._ +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension -import akka.util.Timeout import org.apache.spark.{Logging, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} @@ -38,7 +35,7 @@ import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { import context.dispatcher @@ -537,12 +534,10 @@ 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(classOf[Master], host, boundPort, webUiPort), name = actorName) - val timeoutDuration: FiniteDuration = Duration.create( - System.getProperty("spark.akka.askTimeout", "10").toLong, TimeUnit.SECONDS) - implicit val timeout = Timeout(timeoutDuration) - val respFuture = actor ? RequestWebUIPort // ask pattern - val resp = Await.result(respFuture, timeoutDuration).asInstanceOf[WebUIPortResponse] + val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), actorName) + val timeout = AkkaUtils.askTimeout + val respFuture = actor.ask(RequestWebUIPort)(timeout) + val resp = Await.result(respFuture, timeout).asInstanceOf[WebUIPortResponse] (actorSystem, boundPort, resp.webUIBoundPort) } } 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 3b983c19eb..dbb0cb90f5 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 @@ -17,32 +17,28 @@ package org.apache.spark.deploy.master.ui +import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask - -import scala.concurrent.Await -import scala.concurrent.duration._ - import javax.servlet.http.HttpServletRequest - import net.liftweb.json.JsonAST.JValue -import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.JsonProtocol +import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.ExecutorInfo import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils private[spark] class ApplicationPage(parent: MasterWebUI) { val master = parent.masterActorRef - implicit val timeout = parent.timeout + 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[MasterStateResponse] - val state = Await.result(stateFuture, 30 seconds) + val state = Await.result(stateFuture, timeout) val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) }) @@ -53,7 +49,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { def render(request: HttpServletRequest): Seq[Node] = { val appId = request.getParameter("appId") val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] - val state = Await.result(stateFuture, 30 seconds) + val state = Await.result(stateFuture, timeout) val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) }) 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 65e7a14e7a..4ef762892c 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 @@ -17,37 +17,33 @@ package org.apache.spark.deploy.master.ui -import javax.servlet.http.HttpServletRequest - +import scala.concurrent.Await import scala.xml.Node -import scala.concurrent.Await import akka.pattern.ask -import scala.concurrent.duration._ - +import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue -import org.apache.spark.deploy.DeployWebUI +import org.apache.spark.deploy.{DeployWebUI, JsonProtocol} 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 - implicit val timeout = parent.timeout + val timeout = parent.timeout def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] - val state = Await.result(stateFuture, 30 seconds) + val state = Await.result(stateFuture, timeout) JsonProtocol.writeMasterState(state) } /** Index view listing applications and executors */ def render(request: HttpServletRequest): Seq[Node] = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] - val state = Await.result(stateFuture, 30 seconds) + val state = Await.result(stateFuture, timeout) val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory") val workers = state.workers.sortBy(_.id) 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 a211ce2b42..9ab594b682 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 @@ -17,25 +17,21 @@ package org.apache.spark.deploy.master.ui -import scala.concurrent.duration._ - import javax.servlet.http.HttpServletRequest - import org.eclipse.jetty.server.{Handler, Server} -import org.apache.spark.{Logging} +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 +import org.apache.spark.util.{AkkaUtils, Utils} /** * Web UI server for the standalone master. */ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { - implicit val timeout = Duration.create( - System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + val timeout = AkkaUtils.askTimeout val host = Utils.localHostName() val port = requestedPort 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 1a768d501f..0d59048313 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 @@ -42,13 +42,13 @@ private[spark] class IndexPage(parent: WorkerWebUI) { def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] - val workerState = Await.result(stateFuture, 30 seconds) + val workerState = Await.result(stateFuture, timeout) JsonProtocol.writeWorkerState(workerState) } def render(request: HttpServletRequest): Seq[Node] = { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] - val workerState = Await.result(stateFuture, 30 seconds) + val workerState = Await.result(stateFuture, timeout) val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") val runningExecutorTable = 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 6c18a3c245..40d6bdb3fd 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 @@ -19,17 +19,14 @@ package org.apache.spark.deploy.worker.ui import java.io.File -import scala.concurrent.duration._ - -import akka.util.Timeout import javax.servlet.http.HttpServletRequest +import org.eclipse.jetty.server.{Handler, Server} import org.apache.spark.Logging import org.apache.spark.deploy.worker.Worker import org.apache.spark.ui.{JettyUtils, UIUtils} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.Utils -import org.eclipse.jetty.server.{Handler, Server} +import org.apache.spark.util.{AkkaUtils, Utils} /** * Web UI server for the standalone worker. @@ -37,8 +34,7 @@ import org.eclipse.jetty.server.{Handler, Server} private[spark] 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")) + val timeout = AkkaUtils.askTimeout val host = Utils.localHostName() val port = requestedPort.getOrElse( System.getProperty("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index f5e8766f6d..7e22c843bf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -27,10 +27,10 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.{SparkException, Logging, TaskState} +import org.apache.spark.{Logging, SparkException, TaskState} import org.apache.spark.scheduler.TaskDescription import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{AkkaUtils, Utils} /** * A scheduler backend that waits for coarse grained executors to connect to it through Akka. @@ -47,6 +47,8 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) + private val timeout = AkkaUtils.askTimeout + class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { private val executorActor = new HashMap[String, ActorRef] private val executorAddress = new HashMap[String, Address] @@ -172,10 +174,6 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) } - private val timeout = { - Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") - } - def stopExecutors() { try { if (driverActor != null) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index e05b842476..e1d68ef592 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -18,7 +18,6 @@ package org.apache.spark.storage import scala.concurrent.{Await, Future} -import scala.concurrent.duration._ import scala.concurrent.ExecutionContext.Implicits.global import akka.actor._ @@ -26,15 +25,17 @@ import akka.pattern.ask import org.apache.spark.{Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ +import org.apache.spark.util.AkkaUtils -private[spark] class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection]) extends Logging { +private[spark] +class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection]) extends Logging { val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" - val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + val timeout = AkkaUtils.askTimeout /** Remove a dead executor from the driver actor. This is only called on the driver side. */ def removeExecutor(execId: String) { 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 154a3980e9..21022e1cfb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -21,17 +21,15 @@ import java.util.{HashMap => JHashMap} import scala.collection.mutable import scala.collection.JavaConversions._ +import scala.concurrent.Future +import scala.concurrent.duration._ import akka.actor.{Actor, ActorRef, Cancellable} import akka.pattern.ask -import scala.concurrent.duration._ -import scala.concurrent.Future - import org.apache.spark.{Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.util.Utils - +import org.apache.spark.util.{AkkaUtils, Utils} /** * BlockManagerMasterActor is an actor on the master node to track statuses of @@ -50,8 +48,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { // Mapping from block id to the set of block managers that have the block. private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]] - val akkaTimeout = Duration.create( - System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + private val akkaTimeout = AkkaUtils.askTimeout initLogging() diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index a5446b3fc3..39f422dd6b 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -28,9 +28,6 @@ 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 { - implicit val timeout = Duration.create( - System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") - val indexPage = new IndexPage(this) val rddPage = new RDDPage(this) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 74133cef6c..1c8b51b8bc 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -17,6 +17,8 @@ package org.apache.spark.util +import scala.concurrent.duration.{Duration, FiniteDuration} + import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} import com.typesafe.config.ConfigFactory @@ -84,4 +86,8 @@ private[spark] object AkkaUtils { (actorSystem, boundPort) } + /** Returns the default Spark timeout to use for Akka ask operations. */ + def askTimeout: FiniteDuration = { + Duration.create(System.getProperty("spark.akka.askTimeout", "30").toLong, "seconds") + } } -- cgit v1.2.3 From ec71b445ad0440e84c4b4909e4faf75aba0f13d7 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 18 Dec 2013 23:39:28 -0800 Subject: Minor changes. --- .../org/apache/spark/scheduler/SparkListenerBus.scala | 1 - .../org/apache/spark/streaming/StreamingContext.scala | 10 ++++++---- .../spark/streaming/api/java/JavaStreamingContext.scala | 8 ++++++++ .../apache/spark/streaming/scheduler/BatchInfo.scala | 7 +++---- .../org/apache/spark/streaming/scheduler/Job.scala | 14 ++++---------- .../apache/spark/streaming/scheduler/JobGenerator.scala | 4 ++++ .../apache/spark/streaming/scheduler/JobScheduler.scala | 4 +++- .../spark/streaming/scheduler/StreamingListener.scala | 17 ++++++----------- .../streaming/scheduler/StreamingListenerBus.scala | 2 +- .../apache/spark/streaming/StreamingListenerSuite.scala | 2 +- 10 files changed, 36 insertions(+), 33 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index d5824e7954..85687ea330 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -91,4 +91,3 @@ private[spark] class SparkListenerBus() extends Logging { return true } } - 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 fedbbde80c..41da028a3c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -513,7 +513,10 @@ class StreamingContext private ( graph.addOutputStream(outputStream) } - def addListener(streamingListener: StreamingListener) { + /** Add a [[org.apache.spark.streaming.scheduler.StreamingListener]] object for + * receiving system events related to streaming. + */ + def addStreamingListener(streamingListener: StreamingListener) { scheduler.listenerBus.addListener(streamingListener) } @@ -532,20 +535,19 @@ class StreamingContext private ( * Start the execution of the streams. */ def start() { - validate() + // Get the network input streams val networkInputStreams = graph.getInputStreams().filter(s => s match { case n: NetworkInputDStream[_] => true case _ => false }).map(_.asInstanceOf[NetworkInputDStream[_]]).toArray + // Start the network input tracker (must start before receivers) if (networkInputStreams.length > 0) { - // Start the network input tracker (must start before receivers) networkInputTracker = new NetworkInputTracker(this, networkInputStreams) networkInputTracker.start() } - Thread.sleep(1000) // Start the scheduler 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 80dcf87491..78d318cf27 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 @@ -39,6 +39,7 @@ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => J import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaRDD} import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.scheduler.StreamingListener /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -687,6 +688,13 @@ class JavaStreamingContext(val ssc: StreamingContext) { ssc.remember(duration) } + /** Add a [[org.apache.spark.streaming.scheduler.StreamingListener]] object for + * receiving system events related to streaming. + */ + def addStreamingListener(streamingListener: StreamingListener) { + ssc.addStreamingListener(streamingListener) + } + /** * Starts the execution of the streams. */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala index 798598ad50..88e4af59b7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -19,6 +19,9 @@ package org.apache.spark.streaming.scheduler import org.apache.spark.streaming.Time +/** + * Class having information on completed batches. + */ case class BatchInfo( batchTime: Time, submissionTime: Long, @@ -32,7 +35,3 @@ case class BatchInfo( def totalDelay = schedulingDelay.zip(processingDelay).map(x => x._1 + x._2).headOption } - - - - diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala index bca5e1f1a5..7341bfbc99 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala @@ -17,9 +17,11 @@ package org.apache.spark.streaming.scheduler -import java.util.concurrent.atomic.AtomicLong import org.apache.spark.streaming.Time +/** + * Class representing a Spark computation. It may contain multiple Spark jobs. + */ private[streaming] class Job(val time: Time, func: () => _) { var id: String = _ @@ -36,12 +38,4 @@ class Job(val time: Time, func: () => _) { } override def toString = id -} -/* -private[streaming] -object Job { - val id = new AtomicLong(0) - - def getNewId() = id.getAndIncrement() -} -*/ +} \ No newline at end of file diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 5d3ce9c398..1cd0b9b0a4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -22,6 +22,10 @@ import org.apache.spark.Logging import org.apache.spark.streaming.{Checkpoint, Time, CheckpointWriter} import org.apache.spark.streaming.util.{ManualClock, RecurringTimer, Clock} +/** + * This class generates jobs from DStreams as well as drives checkpointing and cleaning + * up DStream metadata. + */ private[streaming] class JobGenerator(jobScheduler: JobScheduler) extends Logging { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 69930f3b6c..33c5322358 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -23,7 +23,9 @@ import java.util.concurrent.{TimeUnit, ConcurrentHashMap, Executors} import scala.collection.mutable.HashSet import org.apache.spark.streaming._ - +/** + * This class drives the generation of Spark jobs from the DStreams. + */ private[streaming] class JobScheduler(val ssc: StreamingContext) extends Logging { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala index 5647ffab8d..36225e190c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala @@ -50,19 +50,13 @@ trait StreamingListener { * @param numBatchInfos Number of last batches to consider for generating statistics (default: 10) */ class StatsReportListener(numBatchInfos: Int = 10) extends StreamingListener { - - import org.apache.spark - + // Queue containing latest completed batches val batchInfos = new Queue[BatchInfo]() override def onBatchCompleted(batchStarted: StreamingListenerBatchCompleted) { - addToQueue(batchStarted.batchInfo) - printStats() - } - - def addToQueue(newPoint: BatchInfo) { - batchInfos.enqueue(newPoint) + batchInfos.enqueue(batchStarted.batchInfo) if (batchInfos.size > numBatchInfos) batchInfos.dequeue() + printStats() } def printStats() { @@ -71,10 +65,11 @@ class StatsReportListener(numBatchInfos: Int = 10) extends StreamingListener { } def showMillisDistribution(heading: String, getMetric: BatchInfo => Option[Long]) { - spark.scheduler.StatsReportListener.showMillisDistribution(heading, extractDistribution(getMetric)) + org.apache.spark.scheduler.StatsReportListener.showMillisDistribution( + heading, extractDistribution(getMetric)) } def extractDistribution(getMetric: BatchInfo => Option[Long]): Option[Distribution] = { Distribution(batchInfos.flatMap(getMetric(_)).map(_.toDouble)) } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala index 324e491914..110a20f282 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala @@ -78,4 +78,4 @@ private[spark] class StreamingListenerBus() extends Logging { } return true } -} \ No newline at end of file +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 826c839932..16410a21e3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -34,7 +34,7 @@ class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers{ test("basic BatchInfo generation") { val ssc = setupStreams(input, operation) val collector = new BatchInfoCollector - ssc.addListener(collector) + ssc.addStreamingListener(collector) runStreams(ssc, input.size, input.size) val batchInfos = collector.batchInfos batchInfos should have size 4 -- cgit v1.2.3 From a76f53416c0267e8a9816ee4d22fe8e838c4c319 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Thu, 19 Dec 2013 14:38:20 +0200 Subject: Add toString to Java RDD, and __repr__ to Python RDD --- core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala | 2 ++ python/pyspark/rdd.py | 3 +++ 2 files changed, 5 insertions(+) (limited to 'core/src') 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 c47657f512..037cd1c774 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 @@ -125,6 +125,8 @@ JavaRDDLike[T, JavaRDD[T]] { */ def subtract(other: JavaRDD[T], p: Partitioner): JavaRDD[T] = wrapRDD(rdd.subtract(other, p)) + + override def toString = rdd.toString } object JavaRDD { diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 61720dcf1a..7cbc66d3c9 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -54,6 +54,9 @@ class RDD(object): self.ctx = ctx self._jrdd_deserializer = jrdd_deserializer + def __repr__(self): + return self._jrdd.toString() + @property def context(self): """ -- cgit v1.2.3 From d3234f9726db3917af4688ba70933938b078b0bd Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 19 Dec 2013 11:40:34 -0800 Subject: Make collectPartitions take an array of partitions Change the implementation to use runJob instead of PartitionPruningRDD. Also update the unit tests and the python take implementation to use the new interface. --- .../scala/org/apache/spark/api/java/JavaRDDLike.scala | 8 ++++---- .../src/test/scala/org/apache/spark/JavaAPISuite.java | 19 ++++++++++++------- python/pyspark/rdd.py | 7 ++++++- 3 files changed, 22 insertions(+), 12 deletions(-) (limited to 'core/src') 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 1d71875ed1..458d9dcbc3 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 @@ -26,7 +26,7 @@ import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec import org.apache.spark.{SparkContext, Partition, TaskContext} -import org.apache.spark.rdd.{RDD, PartitionPruningRDD} +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} @@ -247,10 +247,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return an array that contains all of the elements in a specific partition of this RDD. */ - def collectPartition(partitionId: Int): JList[T] = { + def collectPartitions(partitionIds: Array[Int]): Array[JList[T]] = { import scala.collection.JavaConversions._ - val partition = new PartitionPruningRDD[T](rdd, _ == partitionId) - new java.util.ArrayList(partition.collect().toSeq) + val res = context.runJob(rdd, (it: Iterator[T]) => it.toArray, partitionIds, true) + res.map(x => new java.util.ArrayList(x.toSeq)).toArray } /** diff --git a/core/src/test/scala/org/apache/spark/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java index 2862ed3019..79913dc718 100644 --- a/core/src/test/scala/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java @@ -899,7 +899,7 @@ public class JavaAPISuite implements Serializable { } @Test - public void collectPartition() { + public void collectPartitions() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); JavaPairRDD rdd2 = rdd1.map(new PairFunction() { @@ -909,20 +909,25 @@ public class JavaAPISuite implements Serializable { } }); - Assert.assertEquals(Arrays.asList(1, 2), rdd1.collectPartition(0)); - Assert.assertEquals(Arrays.asList(3, 4), rdd1.collectPartition(1)); - Assert.assertEquals(Arrays.asList(5, 6, 7), rdd1.collectPartition(2)); + List[] parts = rdd1.collectPartitions(new int[] {0}); + Assert.assertEquals(Arrays.asList(1, 2), parts[0]); + + parts = rdd1.collectPartitions(new int[] {1, 2}); + Assert.assertEquals(Arrays.asList(3, 4), parts[0]); + Assert.assertEquals(Arrays.asList(5, 6, 7), parts[1]); Assert.assertEquals(Arrays.asList(new Tuple2(1, 1), new Tuple2(2, 0)), - rdd2.collectPartition(0)); + rdd2.collectPartitions(new int[] {0})[0]); + + parts = rdd2.collectPartitions(new int[] {1, 2}); Assert.assertEquals(Arrays.asList(new Tuple2(3, 1), new Tuple2(4, 0)), - rdd2.collectPartition(1)); + parts[0]); Assert.assertEquals(Arrays.asList(new Tuple2(5, 1), new Tuple2(6, 0), new Tuple2(7, 1)), - rdd2.collectPartition(2)); + parts[1]); } } diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index d81b7c90c1..7015119551 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -576,8 +576,13 @@ class RDD(object): # Take only up to num elements from each partition we try mapped = self.mapPartitions(takeUpToNum) items = [] + # TODO(shivaram): Similar to the scala implementation, update the take + # method to scan multiple splits based on an estimate of how many elements + # we have per-split. for partition in range(mapped._jrdd.splits().size()): - iterator = mapped._jrdd.collectPartition(partition).iterator() + partitionsToTake = self.ctx._gateway.new_array(self.ctx._jvm.int, 1) + partitionsToTake[0] = partition + iterator = mapped._jrdd.collectPartitions(partitionsToTake)[0].iterator() items.extend(mapped._collect_iterator_through_file(iterator)) if len(items) >= num: break -- cgit v1.2.3 From 9cc3a6d3c0a64b80af77ae358c58d4b29b18c534 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 19 Dec 2013 11:49:17 -0800 Subject: Add comment explaining collectPartitions's use --- core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 2 ++ 1 file changed, 2 insertions(+) (limited to 'core/src') 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 458d9dcbc3..f344804b4c 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 @@ -248,6 +248,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an array that contains all of the elements in a specific partition of this RDD. */ def collectPartitions(partitionIds: Array[Int]): Array[JList[T]] = { + // This is useful for implementing `take` from other language frontends + // like Python where the data is serialized. import scala.collection.JavaConversions._ val res = context.runJob(rdd, (it: Iterator[T]) => it.toArray, partitionIds, true) res.map(x => new java.util.ArrayList(x.toSeq)).toArray -- cgit v1.2.3 From 0647ec97573dc267c7a6b4679fb938b4dfa4fbb6 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 19 Dec 2013 15:10:48 -0800 Subject: Clean up shuffle files once their metadata is gone Previously, we would only clean the in-memory metadata for consolidated shuffle files. Additionally, fixes a bug where the Metadata Cleaner was ignoring type- specific TTLs. --- .../apache/spark/storage/ShuffleBlockManager.scala | 25 +++++++++++++++++++--- .../org/apache/spark/util/MetadataCleaner.scala | 2 +- .../org/apache/spark/util/TimeStampedHashMap.scala | 15 ++++++++++--- 3 files changed, 35 insertions(+), 7 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index e828e1d1c5..212ef6506f 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -70,10 +70,16 @@ class ShuffleBlockManager(blockManager: BlockManager) { * Contains all the state related to a particular shuffle. This includes a pool of unused * ShuffleFileGroups, as well as all ShuffleFileGroups that have been created for the shuffle. */ - private class ShuffleState() { + private class ShuffleState(val numBuckets: Int) { val nextFileId = new AtomicInteger(0) val unusedFileGroups = new ConcurrentLinkedQueue[ShuffleFileGroup]() val allFileGroups = new ConcurrentLinkedQueue[ShuffleFileGroup]() + + /** + * The mapIds of all map tasks completed on this Executor for this shuffle. + * NB: This is only populated if consolidateShuffleFiles is FALSE. We don't need it otherwise. + */ + val completedMapTasks = new ConcurrentLinkedQueue[Int]() } type ShuffleId = Int @@ -84,7 +90,7 @@ class ShuffleBlockManager(blockManager: BlockManager) { def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer) = { new ShuffleWriterGroup { - shuffleStates.putIfAbsent(shuffleId, new ShuffleState()) + shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numBuckets)) private val shuffleState = shuffleStates(shuffleId) private var fileGroup: ShuffleFileGroup = null @@ -109,6 +115,8 @@ class ShuffleBlockManager(blockManager: BlockManager) { fileGroup.recordMapOutput(mapId, offsets) } recycleFileGroup(fileGroup) + } else { + shuffleState.completedMapTasks.add(mapId) } } @@ -154,7 +162,18 @@ class ShuffleBlockManager(blockManager: BlockManager) { } private def cleanup(cleanupTime: Long) { - shuffleStates.clearOldValues(cleanupTime) + shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => { + if (consolidateShuffleFiles) { + for (fileGroup <- state.allFileGroups; file <- fileGroup.files) { + file.delete() + } + } else { + for (mapId <- state.completedMapTasks; reduceId <- 0 until state.numBuckets) { + val blockId = new ShuffleBlockId(shuffleId, mapId, reduceId) + blockManager.diskBlockManager.getFile(blockId).delete() + } + } + }) } } diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 7b41ef89f1..fe56960cbf 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -27,7 +27,7 @@ import org.apache.spark.Logging class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, cleanupFunc: (Long) => Unit) extends Logging { val name = cleanerType.toString - private val delaySeconds = MetadataCleaner.getDelaySeconds + private val delaySeconds = MetadataCleaner.getDelaySeconds(cleanerType) private val periodSeconds = math.max(10, delaySeconds / 10) private val timer = new Timer(name + " cleanup timer", true) diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala index dbff571de9..181ae2fd45 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala @@ -104,19 +104,28 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() with Logging { def toMap: immutable.Map[A, B] = iterator.toMap /** - * Removes old key-value pairs that have timestamp earlier than `threshTime` + * Removes old key-value pairs that have timestamp earlier than `threshTime`, + * calling the supplied function on each such entry before removing. */ - def clearOldValues(threshTime: Long) { + def clearOldValues(threshTime: Long, f: (A, B) => Unit) { val iterator = internalMap.entrySet().iterator() - while(iterator.hasNext) { + while (iterator.hasNext) { val entry = iterator.next() if (entry.getValue._2 < threshTime) { + f(entry.getKey, entry.getValue._1) logDebug("Removing key " + entry.getKey) iterator.remove() } } } + /** + * Removes old key-value pairs that have timestamp earlier than `threshTime` + */ + def clearOldValues(threshTime: Long) { + clearOldValues(threshTime, (_, _) => ()) + } + private def currentTime: Long = System.currentTimeMillis() } -- cgit v1.2.3 From 1ab031eaff7fb2473adb3e909a7a969e9cd28b49 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 19 Dec 2013 17:51:29 -0800 Subject: Extraordinarily minor code/comment cleanup --- .../org/apache/spark/deploy/master/SparkZooKeeperSession.scala | 10 +++++----- .../spark/deploy/master/ZooKeeperLeaderElectionAgent.scala | 4 ++-- 2 files changed, 7 insertions(+), 7 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala index 81e15c534f..5b957fcd5a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala @@ -18,12 +18,12 @@ package org.apache.spark.deploy.master import scala.collection.JavaConversions._ -import scala.concurrent.ops._ -import org.apache.spark.Logging import org.apache.zookeeper._ -import org.apache.zookeeper.data.Stat import org.apache.zookeeper.Watcher.Event.KeeperState +import org.apache.zookeeper.data.Stat + +import org.apache.spark.Logging /** * Provides a Scala-side interface to the standard ZooKeeper client, with the addition of retry @@ -33,7 +33,7 @@ import org.apache.zookeeper.Watcher.Event.KeeperState * informed via zkDown(). * * Additionally, all commands sent to ZooKeeper will be retried until they either fail too many - * times or a semantic exception is thrown (e.g.., "node already exists"). + * times or a semantic exception is thrown (e.g., "node already exists"). */ private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) extends Logging { val ZK_URL = System.getProperty("spark.deploy.zookeeper.url", "") @@ -179,7 +179,7 @@ private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) ext } catch { case e: KeeperException.NoNodeException => throw e case e: KeeperException.NodeExistsException => throw e - case e if n > 0 => + case e: Exception if n > 0 => logError("ZooKeeper exception, " + n + " more retries...", e) Thread.sleep(RETRY_WAIT_MILLIS) retry(fn, n-1) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 7809013e83..7d535b08de 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -21,8 +21,8 @@ import akka.actor.ActorRef import org.apache.zookeeper._ import org.apache.zookeeper.Watcher.Event.EventType -import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.Logging +import org.apache.spark.deploy.master.MasterMessages._ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String) extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging { @@ -105,7 +105,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, mas // We found a different master file pointing to this process. // This can happen in the following two cases: // (1) The master process was restarted on the same node. - // (2) The ZK server died between creating the node and returning the name of the node. + // (2) The ZK server died between creating the file and returning the name of the file. // For this case, we will end up creating a second file, and MUST explicitly delete the // first one, since our ZK session is still open. // Note that this deletion will cause a NodeDeleted event to be fired so we check again for -- cgit v1.2.3 From 4d74b899b7daff74054b70f38cddf7b38fe6c211 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 19 Dec 2013 17:53:41 -0800 Subject: Remove firstApp from the standalone scheduler Master As a lonely child with no one to care for it... we had to put it down. --- .../src/main/scala/org/apache/spark/deploy/master/Master.scala | 10 ---------- 1 file changed, 10 deletions(-) (limited to 'core/src') 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 7b2b1c3327..eebd0794b8 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 @@ -61,8 +61,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val waitingApps = new ArrayBuffer[ApplicationInfo] val completedApps = new ArrayBuffer[ApplicationInfo] - var firstApp: Option[ApplicationInfo] = None - Utils.checkHost(host, "Expected hostname") val masterMetricsSystem = MetricsSystem.createMetricsSystem("master") @@ -441,14 +439,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act idToApp(app.id) = app actorToApp(app.driver) = app addressToApp(appAddress) = app - if (firstApp == None) { - firstApp = Some(app) - } - // TODO: What is firstApp?? Can we remove it? - val workersAlive = workers.filter(_.state == WorkerState.ALIVE).toArray - if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= app.desc.memoryPerSlave)) { - logWarning("Could not find any workers with enough memory for " + firstApp.get.id) - } waitingApps += app } -- cgit v1.2.3 From 6613ab663d42f6b54fe823b06307b8a1005bbb6a Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 19 Dec 2013 17:56:13 -0800 Subject: Fix compiler warning in SparkZooKeeperSession --- .../scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala | 1 + 1 file changed, 1 insertion(+) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala index 5b957fcd5a..6cc7fd2ff4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala @@ -103,6 +103,7 @@ private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) ext connectToZooKeeper() case KeeperState.Disconnected => logWarning("ZooKeeper disconnected, will retry...") + case s => // Do nothing } } } -- cgit v1.2.3 From 61f4bbda0d4e3ecbd8b955232a741231936a25de Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 20 Dec 2013 00:41:47 -0800 Subject: Added tests for PartitionerAwareUnionRDD in the CheckpointSuite. Refactored CheckpointSuite to make the tests simpler and more reliable. Added missing test for ZippedRDD. --- .../spark/rdd/PartitionerAwareUnionRDD.scala | 38 ++- .../org/apache/spark/rdd/RDDCheckpointData.scala | 2 +- .../scala/org/apache/spark/CheckpointSuite.scala | 361 ++++++++++++--------- 3 files changed, 231 insertions(+), 170 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala index 96cf93f99e..995042e590 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -1,16 +1,29 @@ package org.apache.spark.rdd import org.apache.spark.{TaskContext, OneToOneDependency, SparkContext, Partition} +import scala.reflect.ClassTag +import java.io.{ObjectOutputStream, IOException} private[spark] -class PartitionerAwareUnionRDDPartition(val idx: Int, val partitions: Array[Partition]) - extends Partition { +class PartitionerAwareUnionRDDPartition( + @transient val rdds: Seq[RDD[_]], + val idx: Int + ) extends Partition { + var parents = rdds.map(_.partitions(index)).toArray + override val index = idx override def hashCode(): Int = idx + + @throws(classOf[IOException]) + private def writeObject(oos: ObjectOutputStream) { + // Update the reference to parent partition at the time of task serialization + parents = rdds.map(_.partitions(index)).toArray + oos.defaultWriteObject() + } } private[spark] -class PartitionerAwareUnionRDD[T: ClassManifest]( +class PartitionerAwareUnionRDD[T: ClassTag]( sc: SparkContext, var rdds: Seq[RDD[T]] ) extends RDD[T](sc, rdds.map(x => new OneToOneDependency(x))) { @@ -21,17 +34,16 @@ class PartitionerAwareUnionRDD[T: ClassManifest]( override val partitioner = rdds.head.partitioner override def getPartitions: Array[Partition] = { - val numPartitions = rdds.head.partitions.length + val numPartitions = partitioner.get.numPartitions (0 until numPartitions).map(index => { - val parentPartitions = rdds.map(_.partitions(index)).toArray - new PartitionerAwareUnionRDDPartition(index, parentPartitions) + new PartitionerAwareUnionRDDPartition(rdds, index) }).toArray } // Get the location where most of the partitions of parent RDDs are located override def getPreferredLocations(s: Partition): Seq[String] = { logDebug("Getting preferred locations for " + this) - val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].partitions + val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].parents val locations = rdds.zip(parentPartitions).flatMap { case (rdd, part) => { val parentLocations = currPrefLocs(rdd, part) @@ -39,7 +51,6 @@ class PartitionerAwareUnionRDD[T: ClassManifest]( parentLocations } } - if (locations.isEmpty) { Seq.empty } else { @@ -48,18 +59,19 @@ class PartitionerAwareUnionRDD[T: ClassManifest]( } override def compute(s: Partition, context: TaskContext): Iterator[T] = { - val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].partitions + val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].parents rdds.zip(parentPartitions).iterator.flatMap { case (rdd, p) => rdd.iterator(p, context) } } + override def clearDependencies() { + super.clearDependencies() + rdds = null + } + // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) private def currPrefLocs(rdd: RDD[_], part: Partition): Seq[String] = { rdd.context.getPreferredLocs(rdd, part.index).map(tl => tl.host) } } - - - - diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index 3b56e45aa9..fa33a5683a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -40,7 +40,7 @@ private[spark] object CheckpointState extends Enumeration { * manages the post-checkpoint state by providing the updated partitions, iterator and preferred locations * of the checkpointed RDD. */ -private[spark] class RDDCheckpointData[T: ClassTag](rdd: RDD[T]) +private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) extends Logging with Serializable { import CheckpointState._ diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index f25d921d3f..81046af9f3 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -57,15 +57,15 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { } test("RDDs with one-to-one dependencies") { - testCheckpointing(_.map(x => x.toString)) - testCheckpointing(_.flatMap(x => 1 to x)) - testCheckpointing(_.filter(_ % 2 == 0)) - testCheckpointing(_.sample(false, 0.5, 0)) - testCheckpointing(_.glom()) - testCheckpointing(_.mapPartitions(_.map(_.toString))) - testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString)) - testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x)) - testCheckpointing(_.pipe(Seq("cat"))) + testRDD(_.map(x => x.toString)) + testRDD(_.flatMap(x => 1 to x)) + testRDD(_.filter(_ % 2 == 0)) + testRDD(_.sample(false, 0.5, 0)) + testRDD(_.glom()) + testRDD(_.mapPartitions(_.map(_.toString))) + testRDD(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString)) + testRDD(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x)) + testRDD(_.pipe(Seq("cat"))) } test("ParallelCollection") { @@ -97,7 +97,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { } test("ShuffledRDD") { - testCheckpointing(rdd => { + testRDD(rdd => { // Creating ShuffledRDD directly as PairRDDFunctions.combineByKey produces a MapPartitionedRDD new ShuffledRDD[Int, Int, (Int, Int)](rdd.map(x => (x % 2, 1)), partitioner) }) @@ -105,25 +105,17 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { test("UnionRDD") { def otherRDD = sc.makeRDD(1 to 10, 1) - - // Test whether the size of UnionRDDPartitions reduce in size after parent RDD is checkpointed. - // Current implementation of UnionRDD has transient reference to parent RDDs, - // so only the partitions will reduce in serialized size, not the RDD. - testCheckpointing(_.union(otherRDD), false, true) - testParentCheckpointing(_.union(otherRDD), false, true) + testRDD(_.union(otherRDD)) + testRDDPartitions(_.union(otherRDD)) } test("CartesianRDD") { def otherRDD = sc.makeRDD(1 to 10, 1) - testCheckpointing(new CartesianRDD(sc, _, otherRDD)) - - // Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed - // Current implementation of CoalescedRDDPartition has transient reference to parent RDD, - // so only the RDD will reduce in serialized size, not the partitions. - testParentCheckpointing(new CartesianRDD(sc, _, otherRDD), true, false) + testRDD(new CartesianRDD(sc, _, otherRDD)) + testRDDPartitions(new CartesianRDD(sc, _, otherRDD)) // Test that the CartesianRDD updates parent partitions (CartesianRDD.s1/s2) after - // the parent RDD has been checkpointed and parent partitions have been changed to HadoopPartitions. + // the parent RDD has been checkpointed and parent partitions have been changed. // Note that this test is very specific to the current implementation of CartesianRDD. val ones = sc.makeRDD(1 to 100, 10).map(x => x) ones.checkpoint() // checkpoint that MappedRDD @@ -134,23 +126,20 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { val splitAfterCheckpoint = serializeDeserialize(cartesian.partitions.head.asInstanceOf[CartesianPartition]) assert( - (splitAfterCheckpoint.s1 != splitBeforeCheckpoint.s1) && - (splitAfterCheckpoint.s2 != splitBeforeCheckpoint.s2), - "CartesianRDD.parents not updated after parent RDD checkpointed" + (splitAfterCheckpoint.s1.getClass != splitBeforeCheckpoint.s1.getClass) && + (splitAfterCheckpoint.s2.getClass != splitBeforeCheckpoint.s2.getClass), + "CartesianRDD.s1 and CartesianRDD.s2 not updated after parent RDD is checkpointed" ) } test("CoalescedRDD") { - testCheckpointing(_.coalesce(2)) + testRDD(_.coalesce(2)) + testRDDPartitions(_.coalesce(2)) - // Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed - // Current implementation of CoalescedRDDPartition has transient reference to parent RDD, - // so only the RDD will reduce in serialized size, not the partitions. - testParentCheckpointing(_.coalesce(2), true, false) - - // Test that the CoalescedRDDPartition updates parent partitions (CoalescedRDDPartition.parents) after - // the parent RDD has been checkpointed and parent partitions have been changed to HadoopPartitions. - // Note that this test is very specific to the current implementation of CoalescedRDDPartitions + // Test that the CoalescedRDDPartition updates parent partitions (CoalescedRDDPartition.parents) + // after the parent RDD has been checkpointed and parent partitions have been changed. + // Note that this test is very specific to the current implementation of + // CoalescedRDDPartitions. val ones = sc.makeRDD(1 to 100, 10).map(x => x) ones.checkpoint() // checkpoint that MappedRDD val coalesced = new CoalescedRDD(ones, 2) @@ -160,33 +149,78 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { val splitAfterCheckpoint = serializeDeserialize(coalesced.partitions.head.asInstanceOf[CoalescedRDDPartition]) assert( - splitAfterCheckpoint.parents.head != splitBeforeCheckpoint.parents.head, - "CoalescedRDDPartition.parents not updated after parent RDD checkpointed" + splitAfterCheckpoint.parents.head.getClass != splitBeforeCheckpoint.parents.head.getClass, + "CoalescedRDDPartition.parents not updated after parent RDD is checkpointed" ) } test("CoGroupedRDD") { - val longLineageRDD1 = generateLongLineageRDDForCoGroupedRDD() - testCheckpointing(rdd => { + val longLineageRDD1 = generateFatPairRDD() + testRDD(rdd => { CheckpointSuite.cogroup(longLineageRDD1, rdd.map(x => (x % 2, 1)), partitioner) - }, false, true) + }) - val longLineageRDD2 = generateLongLineageRDDForCoGroupedRDD() - testParentCheckpointing(rdd => { + val longLineageRDD2 = generateFatPairRDD() + testRDDPartitions(rdd => { CheckpointSuite.cogroup( longLineageRDD2, sc.makeRDD(1 to 2, 2).map(x => (x % 2, 1)), partitioner) - }, false, true) + }) } test("ZippedRDD") { - testCheckpointing( - rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false) - - // Test whether size of ZippedRDD reduce in size after parent RDD is checkpointed - // Current implementation of ZippedRDDPartitions has transient references to parent RDDs, - // so only the RDD will reduce in serialized size, not the partitions. - testParentCheckpointing( - rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false) + testRDD(rdd => new ZippedRDD(sc, rdd, rdd.map(x => x))) + testRDDPartitions(rdd => new ZippedRDD(sc, rdd, rdd.map(x => x))) + + // Test that the ZippedPartition updates parent partitions + // after the parent RDD has been checkpointed and parent partitions have been changed. + // Note that this test is very specific to the current implementation of ZippedRDD. + val rdd = generateFatRDD() + val zippedRDD = new ZippedRDD(sc, rdd, rdd.map(x => x)) + zippedRDD.rdd1.checkpoint() + zippedRDD.rdd2.checkpoint() + val partitionBeforeCheckpoint = + serializeDeserialize(zippedRDD.partitions.head.asInstanceOf[ZippedPartition[_, _]]) + zippedRDD.count() + val partitionAfterCheckpoint = + serializeDeserialize(zippedRDD.partitions.head.asInstanceOf[ZippedPartition[_, _]]) + assert( + partitionAfterCheckpoint.partition1.getClass != partitionBeforeCheckpoint.partition1.getClass && + partitionAfterCheckpoint.partition2.getClass != partitionBeforeCheckpoint.partition2.getClass, + "ZippedRDD.partition1 and ZippedRDD.partition2 not updated after parent RDD is checkpointed" + ) + } + + test("PartitionerAwareUnionRDD") { + testRDD(rdd => { + new PartitionerAwareUnionRDD[(Int, Int)](sc, Array( + generateFatPairRDD(), + rdd.map(x => (x % 2, 1)).reduceByKey(partitioner, _ + _) + )) + }) + + testRDDPartitions(rdd => { + new PartitionerAwareUnionRDD[(Int, Int)](sc, Array( + generateFatPairRDD(), + rdd.map(x => (x % 2, 1)).reduceByKey(partitioner, _ + _) + )) + }) + + // Test that the PartitionerAwareUnionRDD updates parent partitions + // (PartitionerAwareUnionRDD.parents) after the parent RDD has been checkpointed and parent + // partitions have been changed. Note that this test is very specific to the current + // implementation of PartitionerAwareUnionRDD. + val pairRDD = generateFatPairRDD() + pairRDD.checkpoint() + val unionRDD = new PartitionerAwareUnionRDD(sc, Array(pairRDD)) + val partitionBeforeCheckpoint = serializeDeserialize( + unionRDD.partitions.head.asInstanceOf[PartitionerAwareUnionRDDPartition]) + pairRDD.count() + val partitionAfterCheckpoint = serializeDeserialize( + unionRDD.partitions.head.asInstanceOf[PartitionerAwareUnionRDDPartition]) + assert( + partitionBeforeCheckpoint.parents.head.getClass != partitionAfterCheckpoint.parents.head.getClass, + "PartitionerAwareUnionRDDPartition.parents not updated after parent RDD is checkpointed" + ) } test("CheckpointRDD with zero partitions") { @@ -200,29 +234,32 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { } /** - * Test checkpointing of the final RDD generated by the given operation. By default, - * this method tests whether the size of serialized RDD has reduced after checkpointing or not. - * It can also test whether the size of serialized RDD partitions has reduced after checkpointing or - * not, but this is not done by default as usually the partitions do not refer to any RDD and - * therefore never store the lineage. + * Test checkpointing of the RDD generated by the given operation. It tests whether the + * serialized size of the RDD is reduce after checkpointing or not. This function should be called + * on all RDDs that have a parent RDD (i.e., do not call on ParallelCollection, BlockRDD, etc.). */ - def testCheckpointing[U: ClassTag]( - op: (RDD[Int]) => RDD[U], - testRDDSize: Boolean = true, - testRDDPartitionSize: Boolean = false - ) { + def testRDD[U: ClassTag](op: (RDD[Int]) => RDD[U]) { // Generate the final RDD using given RDD operation - val baseRDD = generateLongLineageRDD() + val baseRDD = generateFatRDD() val operatedRDD = op(baseRDD) val parentRDD = operatedRDD.dependencies.headOption.orNull val rddType = operatedRDD.getClass.getSimpleName val numPartitions = operatedRDD.partitions.length + // Force initialization of all the data structures in RDDs + // Without this, serializing the RDD will give a wrong estimate of the size of the RDD + initializeRdd(operatedRDD) + + val partitionsBeforeCheckpoint = operatedRDD.partitions + // Find serialized sizes before and after the checkpoint - val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD) + logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) + val (rddSizeBeforeCheckpoint, partitionSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD) operatedRDD.checkpoint() val result = operatedRDD.collect() - val (rddSizeAfterCheckpoint, splitSizeAfterCheckpoint) = getSerializedSizes(operatedRDD) + operatedRDD.collect() // force re-initialization of post-checkpoint lazy variables + val (rddSizeAfterCheckpoint, partitionSizeAfterCheckpoint) = getSerializedSizes(operatedRDD) + logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) // Test whether the checkpoint file has been created assert(sc.checkpointFile[U](operatedRDD.getCheckpointFile.get).collect() === result) @@ -230,6 +267,9 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { // Test whether dependencies have been changed from its earlier parent RDD assert(operatedRDD.dependencies.head.rdd != parentRDD) + // Test whether the partitions have been changed from its earlier partitions + assert(operatedRDD.partitions.toList != partitionsBeforeCheckpoint.toList) + // Test whether the partitions have been changed to the new Hadoop partitions assert(operatedRDD.partitions.toList === operatedRDD.checkpointData.get.getPartitions.toList) @@ -239,122 +279,72 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { // Test whether the data in the checkpointed RDD is same as original assert(operatedRDD.collect() === result) - // Test whether serialized size of the RDD has reduced. If the RDD - // does not have any dependency to another RDD (e.g., ParallelCollection, - // ShuffleRDD with ShuffleDependency), it may not reduce in size after checkpointing. - if (testRDDSize) { - logInfo("Size of " + rddType + - "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]") - assert( - rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint, - "Size of " + rddType + " did not reduce after checkpointing " + - "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]" - ) - } + // Test whether serialized size of the RDD has reduced. + logInfo("Size of " + rddType + + " [" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]") + assert( + rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint, + "Size of " + rddType + " did not reduce after checkpointing " + + " [" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]" + ) - // Test whether serialized size of the partitions has reduced. If the partitions - // do not have any non-transient reference to another RDD or another RDD's partitions, it - // does not refer to a lineage and therefore may not reduce in size after checkpointing. - // However, if the original partitions before checkpointing do refer to a parent RDD, the partitions - // must be forgotten after checkpointing (to remove all reference to parent RDDs) and - // replaced with the HadooPartitions of the checkpointed RDD. - if (testRDDPartitionSize) { - logInfo("Size of " + rddType + " partitions " - + "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]") - assert( - splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint, - "Size of " + rddType + " partitions did not reduce after checkpointing " + - "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]" - ) - } } /** * Test whether checkpointing of the parent of the generated RDD also * truncates the lineage or not. Some RDDs like CoGroupedRDD hold on to its parent * RDDs partitions. So even if the parent RDD is checkpointed and its partitions changed, - * this RDD will remember the partitions and therefore potentially the whole lineage. + * the generated RDD will remember the partitions and therefore potentially the whole lineage. + * This function should be called only those RDD whose partitions refer to parent RDD's + * partitions (i.e., do not call it on simple RDD like MappedRDD). + * */ - def testParentCheckpointing[U: ClassTag]( - op: (RDD[Int]) => RDD[U], - testRDDSize: Boolean, - testRDDPartitionSize: Boolean - ) { + def testRDDPartitions[U: ClassTag](op: (RDD[Int]) => RDD[U]) { // Generate the final RDD using given RDD operation - val baseRDD = generateLongLineageRDD() + val baseRDD = generateFatRDD() val operatedRDD = op(baseRDD) - val parentRDD = operatedRDD.dependencies.head.rdd + val parentRDDs = operatedRDD.dependencies.map(_.rdd) val rddType = operatedRDD.getClass.getSimpleName - val parentRDDType = parentRDD.getClass.getSimpleName - // Get the partitions and dependencies of the parent in case they're lazily computed - parentRDD.dependencies - parentRDD.partitions + // Force initialization of all the data structures in RDDs + // Without this, serializing the RDD will give a wrong estimate of the size of the RDD + initializeRdd(operatedRDD) // Find serialized sizes before and after the checkpoint - val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD) - parentRDD.checkpoint() // checkpoint the parent RDD, not the generated one - val result = operatedRDD.collect() - val (rddSizeAfterCheckpoint, splitSizeAfterCheckpoint) = getSerializedSizes(operatedRDD) + logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) + val (rddSizeBeforeCheckpoint, partitionSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD) + parentRDDs.foreach(_.checkpoint()) // checkpoint the parent RDD, not the generated one + val result = operatedRDD.collect() // force checkpointing + operatedRDD.collect() // force re-initialization of post-checkpoint lazy variables + val (rddSizeAfterCheckpoint, partitionSizeAfterCheckpoint) = getSerializedSizes(operatedRDD) + logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) // Test whether the data in the checkpointed RDD is same as original assert(operatedRDD.collect() === result) - // Test whether serialized size of the RDD has reduced because of its parent being - // checkpointed. If this RDD or its parent RDD do not have any dependency - // to another RDD (e.g., ParallelCollection, ShuffleRDD with ShuffleDependency), it may - // not reduce in size after checkpointing. - if (testRDDSize) { - assert( - rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint, - "Size of " + rddType + " did not reduce after checkpointing parent " + parentRDDType + - "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]" - ) - } - - // Test whether serialized size of the partitions has reduced because of its parent being - // checkpointed. If the partitions do not have any non-transient reference to another RDD - // or another RDD's partitions, it does not refer to a lineage and therefore may not reduce - // in size after checkpointing. However, if the partitions do refer to the *partitions* of a parent - // RDD, then these partitions must update reference to the parent RDD partitions as the parent RDD's - // partitions must have changed after checkpointing. - if (testRDDPartitionSize) { - assert( - splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint, - "Size of " + rddType + " partitions did not reduce after checkpointing parent " + parentRDDType + - "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]" - ) - } - + // Test whether serialized size of the partitions has reduced + logInfo("Size of partitions of " + rddType + + " [" + partitionSizeBeforeCheckpoint + " --> " + partitionSizeAfterCheckpoint + "]") + assert( + partitionSizeAfterCheckpoint < partitionSizeBeforeCheckpoint, + "Size of " + rddType + " partitions did not reduce after checkpointing parent RDDs" + + " [" + partitionSizeBeforeCheckpoint + " --> " + partitionSizeAfterCheckpoint + "]" + ) } /** - * Generate an RDD with a long lineage of one-to-one dependencies. + * Generate an RDD such that both the RDD and its partitions have large size. */ - def generateLongLineageRDD(): RDD[Int] = { - var rdd = sc.makeRDD(1 to 100, 4) - for (i <- 1 to 50) { - rdd = rdd.map(x => x + 1) - } - rdd + def generateFatRDD(): RDD[Int] = { + new FatRDD(sc.makeRDD(1 to 100, 4)).map(x => x) } /** - * Generate an RDD with a long lineage specifically for CoGroupedRDD. - * A CoGroupedRDD can have a long lineage only one of its parents have a long lineage - * and narrow dependency with this RDD. This method generate such an RDD by a sequence - * of cogroups and mapValues which creates a long lineage of narrow dependencies. + * Generate an pair RDD (with partitioner) such that both the RDD and its partitions + * have large size. */ - def generateLongLineageRDDForCoGroupedRDD() = { - val add = (x: (Seq[Int], Seq[Int])) => (x._1 ++ x._2).reduce(_ + _) - - def ones: RDD[(Int, Int)] = sc.makeRDD(1 to 2, 2).map(x => (x % 2, 1)).reduceByKey(partitioner, _ + _) - - var cogrouped: RDD[(Int, (Seq[Int], Seq[Int]))] = ones.cogroup(ones) - for(i <- 1 to 10) { - cogrouped = cogrouped.mapValues(add).cogroup(ones) - } - cogrouped.mapValues(add) + def generateFatPairRDD() = { + new FatPairRDD(sc.makeRDD(1 to 100, 4), partitioner).mapValues(x => x) } /** @@ -362,8 +352,26 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { * upon checkpointing. Ignores the checkpointData field, which may grow when we checkpoint. */ def getSerializedSizes(rdd: RDD[_]): (Int, Int) = { - (Utils.serialize(rdd).length - Utils.serialize(rdd.checkpointData).length, - Utils.serialize(rdd.partitions).length) + val rddSize = Utils.serialize(rdd).size + val rddCpDataSize = Utils.serialize(rdd.checkpointData).size + val rddPartitionSize = Utils.serialize(rdd.partitions).size + val rddDependenciesSize = Utils.serialize(rdd.dependencies).size + + // Print detailed size, helps in debugging + logInfo("Serialized sizes of " + rdd + + ": RDD = " + rddSize + + ", RDD checkpoint data = " + rddCpDataSize + + ", RDD partitions = " + rddPartitionSize + + ", RDD dependencies = " + rddDependenciesSize + ) + // this makes sure that serializing the RDD's checkpoint data does not + // serialize the whole RDD as well + assert( + rddSize > rddCpDataSize, + "RDD's checkpoint data (" + rddCpDataSize + ") is equal or larger than the " + + "whole RDD with checkpoint data (" + rddSize + ")" + ) + (rddSize - rddCpDataSize, rddPartitionSize) } /** @@ -375,8 +383,49 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { val bytes = Utils.serialize(obj) Utils.deserialize[T](bytes) } + + /** + * Recursively force the initialization of the all members of an RDD and it parents. + */ + def initializeRdd(rdd: RDD[_]) { + rdd.partitions // forces the + rdd.dependencies.map(_.rdd).foreach(initializeRdd(_)) + } } +/** RDD partition that has large serialized size. */ +class FatPartition(val partition: Partition) extends Partition { + val bigData = new Array[Byte](10000) + def index: Int = partition.index +} + +/** RDD that has large serialized size. */ +class FatRDD(parent: RDD[Int]) extends RDD[Int](parent) { + val bigData = new Array[Byte](100000) + + protected def getPartitions: Array[Partition] = { + parent.partitions.map(p => new FatPartition(p)) + } + + def compute(split: Partition, context: TaskContext): Iterator[Int] = { + parent.compute(split.asInstanceOf[FatPartition].partition, context) + } +} + +/** Pair RDD that has large serialized size. */ +class FatPairRDD(parent: RDD[Int], _partitioner: Partitioner) extends RDD[(Int, Int)](parent) { + val bigData = new Array[Byte](100000) + + protected def getPartitions: Array[Partition] = { + parent.partitions.map(p => new FatPartition(p)) + } + + @transient override val partitioner = Some(_partitioner) + + def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = { + parent.compute(split.asInstanceOf[FatPartition].partition, context).map(x => (x, x)) + } +} object CheckpointSuite { // This is a custom cogroup function that does not use mapValues like -- cgit v1.2.3 From 30186aa2648f90d0ad4e312d28e99c9378ea317a Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Fri, 20 Dec 2013 14:58:04 -0800 Subject: Renamed ClusterScheduler to TaskSchedulerImpl --- .../main/scala/org/apache/spark/SparkContext.scala | 20 +- .../apache/spark/scheduler/ClusterScheduler.scala | 473 --------------------- .../apache/spark/scheduler/TaskResultGetter.scala | 2 +- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 473 +++++++++++++++++++++ .../apache/spark/scheduler/TaskSetManager.scala | 2 +- .../cluster/CoarseGrainedSchedulerBackend.scala | 4 +- .../scheduler/cluster/SimrSchedulerBackend.scala | 4 +- .../cluster/SparkDeploySchedulerBackend.scala | 4 +- .../mesos/CoarseMesosSchedulerBackend.scala | 4 +- .../cluster/mesos/MesosSchedulerBackend.scala | 4 +- .../spark/scheduler/local/LocalBackend.scala | 6 +- .../spark/SparkContextSchedulerCreationSuite.scala | 6 +- .../spark/scheduler/ClusterSchedulerSuite.scala | 10 +- .../spark/scheduler/TaskResultGetterSuite.scala | 6 +- .../spark/scheduler/TaskSetManagerSuite.scala | 2 +- 15 files changed, 510 insertions(+), 510 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 663b473e5d..ad3337d94c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1044,25 +1044,25 @@ object SparkContext { master match { case "local" => - val scheduler = new ClusterScheduler(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) + val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalBackend(scheduler, 1) scheduler.initialize(backend) scheduler case LOCAL_N_REGEX(threads) => - val scheduler = new ClusterScheduler(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) + val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalBackend(scheduler, threads.toInt) scheduler.initialize(backend) scheduler case LOCAL_N_FAILURES_REGEX(threads, maxFailures) => - val scheduler = new ClusterScheduler(sc, maxFailures.toInt, isLocal = true) + val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true) val backend = new LocalBackend(scheduler, threads.toInt) scheduler.initialize(backend) scheduler case SPARK_REGEX(sparkUrl) => - val scheduler = new ClusterScheduler(sc) + val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName) scheduler.initialize(backend) @@ -1077,7 +1077,7 @@ object SparkContext { memoryPerSlaveInt, SparkContext.executorMemoryRequested)) } - val scheduler = new ClusterScheduler(sc) + val scheduler = new TaskSchedulerImpl(sc) val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val masterUrls = localCluster.start() @@ -1092,7 +1092,7 @@ object SparkContext { val scheduler = try { val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler") val cons = clazz.getConstructor(classOf[SparkContext]) - cons.newInstance(sc).asInstanceOf[ClusterScheduler] + cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl] } catch { // TODO: Enumerate the exact reasons why it can fail // But irrespective of it, it means we cannot proceed ! @@ -1108,7 +1108,7 @@ object SparkContext { val scheduler = try { val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") val cons = clazz.getConstructor(classOf[SparkContext]) - cons.newInstance(sc).asInstanceOf[ClusterScheduler] + cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl] } catch { case th: Throwable => { @@ -1118,7 +1118,7 @@ object SparkContext { val backend = try { val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend") - val cons = clazz.getConstructor(classOf[ClusterScheduler], classOf[SparkContext]) + val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] } catch { case th: Throwable => { @@ -1131,7 +1131,7 @@ object SparkContext { case mesosUrl @ MESOS_REGEX(_) => MesosNativeLibrary.load() - val scheduler = new ClusterScheduler(sc) + val scheduler = new TaskSchedulerImpl(sc) val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { @@ -1143,7 +1143,7 @@ object SparkContext { scheduler case SIMR_REGEX(simrUrl) => - val scheduler = new ClusterScheduler(sc) + val scheduler = new TaskSchedulerImpl(sc) val backend = new SimrSchedulerBackend(scheduler, sc, simrUrl) scheduler.initialize(backend) scheduler diff --git a/core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala deleted file mode 100644 index 1ad735bc04..0000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala +++ /dev/null @@ -1,473 +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.spark.scheduler - -import java.nio.ByteBuffer -import java.util.concurrent.atomic.AtomicLong -import java.util.{TimerTask, Timer} - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import scala.concurrent.duration._ - -import org.apache.spark._ -import org.apache.spark.TaskState.TaskState -import org.apache.spark.scheduler.SchedulingMode.SchedulingMode - -/** - * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. - * It can also work with a local setup by using a LocalBackend and setting isLocal to true. - * It handles common logic, like determining a scheduling order across jobs, waking up to launch - * speculative tasks, etc. - * - * Clients should first call initialize() and start(), then submit task sets through the - * runTasks method. - * - * 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, - val maxTaskFailures : Int = System.getProperty("spark.task.maxFailures", "4").toInt, - isLocal: Boolean = false) extends TaskScheduler 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 - - // TaskSetManagers are not thread safe, so any access to one should be synchronized - // on this class. - val activeTaskSets = new HashMap[String, TaskSetManager] - - val taskIdToTaskSetId = new HashMap[Long, String] - val taskIdToExecutorId = new HashMap[Long, String] - val taskSetTaskIds = new HashMap[String, HashSet[Long]] - - @volatile private var hasReceivedTask = false - @volatile private var hasLaunchedTask = false - private val starvationTimer = new Timer(true) - - // Incrementing task IDs - val nextTaskId = new AtomicLong(0) - - // Which executor IDs we have executors on - val activeExecutorIds = new 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 executorsByHost = new HashMap[String, HashSet[String]] - - private val executorIdToHost = new HashMap[String, String] - - // Listener object to pass upcalls into - var dagScheduler: DAGScheduler = null - - var backend: SchedulerBackend = null - - val mapOutputTracker = SparkEnv.get.mapOutputTracker - - var schedulableBuilder: SchedulableBuilder = null - var rootPool: Pool = null - // default scheduler is FIFO - val schedulingMode: SchedulingMode = SchedulingMode.withName( - System.getProperty("spark.scheduler.mode", "FIFO")) - - // This is a var so that we can reset it for testing purposes. - private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) - - override def setDAGScheduler(dagScheduler: DAGScheduler) { - this.dagScheduler = dagScheduler - } - - def initialize(backend: SchedulerBackend) { - this.backend = backend - // temporarily set rootPool name to empty - rootPool = new Pool("", schedulingMode, 0, 0) - schedulableBuilder = { - schedulingMode match { - case SchedulingMode.FIFO => - new FIFOSchedulableBuilder(rootPool) - case SchedulingMode.FAIR => - new FairSchedulableBuilder(rootPool) - } - } - schedulableBuilder.buildPools() - } - - def newTaskId(): Long = nextTaskId.getAndIncrement() - - override def start() { - backend.start() - - if (!isLocal && System.getProperty("spark.speculation", "false").toBoolean) { - logInfo("Starting speculative execution thread") - import sc.env.actorSystem.dispatcher - sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, - SPECULATION_INTERVAL milliseconds) { - checkSpeculatableTasks() - } - } - } - - override def submitTasks(taskSet: TaskSet) { - val tasks = taskSet.tasks - logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks") - this.synchronized { - val manager = new TaskSetManager(this, taskSet, maxTaskFailures) - activeTaskSets(taskSet.id) = manager - schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) - taskSetTaskIds(taskSet.id) = new HashSet[Long]() - - if (!isLocal && !hasReceivedTask) { - starvationTimer.scheduleAtFixedRate(new TimerTask() { - override def run() { - if (!hasLaunchedTask) { - logWarning("Initial job has not accepted any resources; " + - "check your cluster UI to ensure that workers are registered " + - "and have sufficient memory") - } else { - this.cancel() - } - } - }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) - } - hasReceivedTask = true - } - backend.reviveOffers() - } - - override def cancelTasks(stageId: Int): Unit = synchronized { - logInfo("Cancelling stage " + stageId) - activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) => - // There are two possible cases here: - // 1. The task set manager has been created and some tasks have been scheduled. - // In this case, send a kill signal to the executors to kill the task and then abort - // the stage. - // 2. The task set manager has been created but no tasks has been scheduled. In this case, - // simply abort the stage. - val taskIds = taskSetTaskIds(tsm.taskSet.id) - if (taskIds.size > 0) { - taskIds.foreach { tid => - val execId = taskIdToExecutorId(tid) - backend.killTask(tid, execId) - } - } - logInfo("Stage %d was cancelled".format(stageId)) - tsm.removeAllRunningTasks() - taskSetFinished(tsm) - } - } - - def taskSetFinished(manager: TaskSetManager): Unit = synchronized { - // Check to see if the given task set has been removed. This is possible in the case of - // multiple unrecoverable task failures (e.g. if the entire task set is killed when it has - // more than one running tasks). - if (activeTaskSets.contains(manager.taskSet.id)) { - activeTaskSets -= manager.taskSet.id - manager.parent.removeSchedulable(manager) - logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) - taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) - taskIdToExecutorId --= taskSetTaskIds(manager.taskSet.id) - taskSetTaskIds.remove(manager.taskSet.id) - } - } - - /** - * Called by cluster manager to offer resources on slaves. We respond by asking our active task - * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so - * that tasks are balanced across the cluster. - */ - def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = synchronized { - 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) - } - } - - // 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 sortedTaskSets = rootPool.getSortedTaskSetQueue() - for (taskSet <- sortedTaskSets) { - logDebug("parentName: %s, name: %s, runningTasks: %s".format( - 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 (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 <- taskSet.resourceOffer(execId, host, availableCpus(i), maxLocality)) { - tasks(i) += task - val tid = task.taskId - taskIdToTaskSetId(tid) = taskSet.taskSet.id - taskSetTaskIds(taskSet.taskSet.id) += tid - taskIdToExecutorId(tid) = execId - activeExecutorIds += execId - executorsByHost(host) += execId - availableCpus(i) -= 1 - launchedTask = true - } - } - } while (launchedTask) - } - - if (tasks.size > 0) { - hasLaunchedTask = true - } - return tasks - } - - def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { - var failedExecutor: Option[String] = None - synchronized { - try { - if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) { - // We lost this entire executor, so remember that it's gone - val execId = taskIdToExecutorId(tid) - if (activeExecutorIds.contains(execId)) { - removeExecutor(execId) - failedExecutor = Some(execId) - } - } - taskIdToTaskSetId.get(tid) match { - case Some(taskSetId) => - if (TaskState.isFinished(state)) { - taskIdToTaskSetId.remove(tid) - if (taskSetTaskIds.contains(taskSetId)) { - taskSetTaskIds(taskSetId) -= tid - } - taskIdToExecutorId.remove(tid) - } - activeTaskSets.get(taskSetId).foreach { taskSet => - if (state == TaskState.FINISHED) { - taskSet.removeRunningTask(tid) - taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) - } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { - taskSet.removeRunningTask(tid) - taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData) - } - } - case None => - logInfo("Ignoring update from TID " + tid + " because its task set is gone") - } - } catch { - case e: Exception => logError("Exception in statusUpdate", e) - } - } - // Update the DAGScheduler without holding a lock on this, since that can deadlock - if (failedExecutor != None) { - dagScheduler.executorLost(failedExecutor.get) - backend.reviveOffers() - } - } - - def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long) { - taskSetManager.handleTaskGettingResult(tid) - } - - def handleSuccessfulTask( - taskSetManager: TaskSetManager, - tid: Long, - taskResult: DirectTaskResult[_]) = synchronized { - taskSetManager.handleSuccessfulTask(tid, taskResult) - } - - def handleFailedTask( - taskSetManager: TaskSetManager, - tid: Long, - taskState: TaskState, - reason: Option[TaskEndReason]) = synchronized { - taskSetManager.handleFailedTask(tid, taskState, reason) - if (taskState != TaskState.KILLED) { - // Need to revive offers again now that the task set manager state has been updated to - // reflect failed tasks that need to be re-run. - backend.reviveOffers() - } - } - - def error(message: String) { - synchronized { - if (activeTaskSets.size > 0) { - // Have each task set throw a SparkException with the error - for ((taskSetId, manager) <- activeTaskSets) { - try { - manager.error(message) - } catch { - case e: Exception => logError("Exception in error callback", e) - } - } - } else { - // No task sets are active but we still got an error. Just exit since this - // must mean the error is during registration. - // It might be good to do something smarter here in the future. - logError("Exiting due to error from cluster scheduler: " + message) - System.exit(1) - } - } - } - - override def stop() { - if (backend != null) { - backend.stop() - } - if (taskResultGetter != null) { - taskResultGetter.stop() - } - - // sleeping for an arbitrary 5 seconds : to ensure that messages are sent out. - // TODO: Do something better ! - Thread.sleep(5000L) - } - - override def defaultParallelism() = backend.defaultParallelism() - - // Check for speculatable tasks in all our active jobs. - def checkSpeculatableTasks() { - var shouldRevive = false - synchronized { - shouldRevive = rootPool.checkSpeculatableTasks() - } - if (shouldRevive) { - backend.reviveOffers() - } - } - - // Check for pending tasks in all our active jobs. - def hasPendingTasks: Boolean = { - synchronized { - rootPool.hasPendingTasks() - } - } - - def executorLost(executorId: String, reason: ExecutorLossReason) { - var failedExecutor: Option[String] = None - - synchronized { - if (activeExecutorIds.contains(executorId)) { - val hostPort = executorIdToHost(executorId) - logError("Lost executor %s on %s: %s".format(executorId, hostPort, reason)) - removeExecutor(executorId) - failedExecutor = Some(executorId) - } else { - // We may get multiple executorLost() calls with different loss reasons. For example, one - // may be triggered by a dropped connection from the slave while another may be a report - // of executor termination from Mesos. We produce log messages for both so we eventually - // report the termination reason. - logError("Lost an executor " + executorId + " (already removed): " + reason) - } - } - // Call dagScheduler.executorLost without holding the lock on this to prevent deadlock - if (failedExecutor != None) { - dagScheduler.executorLost(failedExecutor.get) - backend.reviveOffers() - } - } - - /** Remove an executor from all our data structures and mark it as lost */ - private def removeExecutor(executorId: String) { - activeExecutorIds -= executorId - val host = executorIdToHost(executorId) - val execs = executorsByHost.getOrElse(host, new HashSet) - execs -= executorId - if (execs.isEmpty) { - executorsByHost -= host - } - executorIdToHost -= executorId - rootPool.executorLost(executorId, host) - } - - def executorGained(execId: String, host: String) { - dagScheduler.executorGained(execId, host) - } - - def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized { - executorsByHost.get(host).map(_.toSet) - } - - def hasExecutorsAliveOnHost(host: String): Boolean = synchronized { - executorsByHost.contains(host) - } - - def isExecutorAlive(execId: String): Boolean = synchronized { - activeExecutorIds.contains(execId) - } - - // By default, rack is unknown - def getRackForHost(value: String): Option[String] = None -} - - -private[spark] object ClusterScheduler { - /** - * 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 - - // order keyList based on population of value in map - val keyList = _keyList.sortWith( - (left, right) => map(left).size > map(right).size - ) - - val retval = new ArrayBuffer[T](keyList.size * 2) - var index = 0 - var found = true - - while (found) { - found = false - for (key <- keyList) { - val containerList: ArrayBuffer[T] = map.get(key).getOrElse(null) - assert(containerList != null) - // Get the index'th entry for this host - if present - if (index < containerList.size){ - retval += containerList.apply(index) - found = true - } - } - index += 1 - } - - retval.toList - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 7b5543e222..89102720fa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.Utils /** * Runs a thread pool that deserializes and remotely fetches (if necessary) task results. */ -private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler) +private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedulerImpl) extends Logging { private val THREADS = System.getProperty("spark.resultGetter.threads", "4").toInt private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala new file mode 100644 index 0000000000..7409168f7b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -0,0 +1,473 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicLong +import java.util.{TimerTask, Timer} + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet +import scala.concurrent.duration._ + +import org.apache.spark._ +import org.apache.spark.TaskState.TaskState +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode + +/** + * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. + * It can also work with a local setup by using a LocalBackend and setting isLocal to true. + * It handles common logic, like determining a scheduling order across jobs, waking up to launch + * speculative tasks, etc. + * + * Clients should first call initialize() and start(), then submit task sets through the + * runTasks method. + * + * 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 TaskSchedulerImpl( + val sc: SparkContext, + val maxTaskFailures : Int = System.getProperty("spark.task.maxFailures", "4").toInt, + isLocal: Boolean = false) extends TaskScheduler 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 + + // TaskSetManagers are not thread safe, so any access to one should be synchronized + // on this class. + val activeTaskSets = new HashMap[String, TaskSetManager] + + val taskIdToTaskSetId = new HashMap[Long, String] + val taskIdToExecutorId = new HashMap[Long, String] + val taskSetTaskIds = new HashMap[String, HashSet[Long]] + + @volatile private var hasReceivedTask = false + @volatile private var hasLaunchedTask = false + private val starvationTimer = new Timer(true) + + // Incrementing task IDs + val nextTaskId = new AtomicLong(0) + + // Which executor IDs we have executors on + val activeExecutorIds = new 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 executorsByHost = new HashMap[String, HashSet[String]] + + private val executorIdToHost = new HashMap[String, String] + + // Listener object to pass upcalls into + var dagScheduler: DAGScheduler = null + + var backend: SchedulerBackend = null + + val mapOutputTracker = SparkEnv.get.mapOutputTracker + + var schedulableBuilder: SchedulableBuilder = null + var rootPool: Pool = null + // default scheduler is FIFO + val schedulingMode: SchedulingMode = SchedulingMode.withName( + System.getProperty("spark.scheduler.mode", "FIFO")) + + // This is a var so that we can reset it for testing purposes. + private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) + + override def setDAGScheduler(dagScheduler: DAGScheduler) { + this.dagScheduler = dagScheduler + } + + def initialize(backend: SchedulerBackend) { + this.backend = backend + // temporarily set rootPool name to empty + rootPool = new Pool("", schedulingMode, 0, 0) + schedulableBuilder = { + schedulingMode match { + case SchedulingMode.FIFO => + new FIFOSchedulableBuilder(rootPool) + case SchedulingMode.FAIR => + new FairSchedulableBuilder(rootPool) + } + } + schedulableBuilder.buildPools() + } + + def newTaskId(): Long = nextTaskId.getAndIncrement() + + override def start() { + backend.start() + + if (!isLocal && System.getProperty("spark.speculation", "false").toBoolean) { + logInfo("Starting speculative execution thread") + import sc.env.actorSystem.dispatcher + sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, + SPECULATION_INTERVAL milliseconds) { + checkSpeculatableTasks() + } + } + } + + override def submitTasks(taskSet: TaskSet) { + val tasks = taskSet.tasks + logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks") + this.synchronized { + val manager = new TaskSetManager(this, taskSet, maxTaskFailures) + activeTaskSets(taskSet.id) = manager + schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) + taskSetTaskIds(taskSet.id) = new HashSet[Long]() + + if (!isLocal && !hasReceivedTask) { + starvationTimer.scheduleAtFixedRate(new TimerTask() { + override def run() { + if (!hasLaunchedTask) { + logWarning("Initial job has not accepted any resources; " + + "check your cluster UI to ensure that workers are registered " + + "and have sufficient memory") + } else { + this.cancel() + } + } + }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) + } + hasReceivedTask = true + } + backend.reviveOffers() + } + + override def cancelTasks(stageId: Int): Unit = synchronized { + logInfo("Cancelling stage " + stageId) + activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) => + // There are two possible cases here: + // 1. The task set manager has been created and some tasks have been scheduled. + // In this case, send a kill signal to the executors to kill the task and then abort + // the stage. + // 2. The task set manager has been created but no tasks has been scheduled. In this case, + // simply abort the stage. + val taskIds = taskSetTaskIds(tsm.taskSet.id) + if (taskIds.size > 0) { + taskIds.foreach { tid => + val execId = taskIdToExecutorId(tid) + backend.killTask(tid, execId) + } + } + logInfo("Stage %d was cancelled".format(stageId)) + tsm.removeAllRunningTasks() + taskSetFinished(tsm) + } + } + + def taskSetFinished(manager: TaskSetManager): Unit = synchronized { + // Check to see if the given task set has been removed. This is possible in the case of + // multiple unrecoverable task failures (e.g. if the entire task set is killed when it has + // more than one running tasks). + if (activeTaskSets.contains(manager.taskSet.id)) { + activeTaskSets -= manager.taskSet.id + manager.parent.removeSchedulable(manager) + logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) + taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) + taskIdToExecutorId --= taskSetTaskIds(manager.taskSet.id) + taskSetTaskIds.remove(manager.taskSet.id) + } + } + + /** + * Called by cluster manager to offer resources on slaves. We respond by asking our active task + * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so + * that tasks are balanced across the cluster. + */ + def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = synchronized { + 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) + } + } + + // 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 sortedTaskSets = rootPool.getSortedTaskSetQueue() + for (taskSet <- sortedTaskSets) { + logDebug("parentName: %s, name: %s, runningTasks: %s".format( + 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 (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 <- taskSet.resourceOffer(execId, host, availableCpus(i), maxLocality)) { + tasks(i) += task + val tid = task.taskId + taskIdToTaskSetId(tid) = taskSet.taskSet.id + taskSetTaskIds(taskSet.taskSet.id) += tid + taskIdToExecutorId(tid) = execId + activeExecutorIds += execId + executorsByHost(host) += execId + availableCpus(i) -= 1 + launchedTask = true + } + } + } while (launchedTask) + } + + if (tasks.size > 0) { + hasLaunchedTask = true + } + return tasks + } + + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + var failedExecutor: Option[String] = None + synchronized { + try { + if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) { + // We lost this entire executor, so remember that it's gone + val execId = taskIdToExecutorId(tid) + if (activeExecutorIds.contains(execId)) { + removeExecutor(execId) + failedExecutor = Some(execId) + } + } + taskIdToTaskSetId.get(tid) match { + case Some(taskSetId) => + if (TaskState.isFinished(state)) { + taskIdToTaskSetId.remove(tid) + if (taskSetTaskIds.contains(taskSetId)) { + taskSetTaskIds(taskSetId) -= tid + } + taskIdToExecutorId.remove(tid) + } + activeTaskSets.get(taskSetId).foreach { taskSet => + if (state == TaskState.FINISHED) { + taskSet.removeRunningTask(tid) + taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) + } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { + taskSet.removeRunningTask(tid) + taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData) + } + } + case None => + logInfo("Ignoring update from TID " + tid + " because its task set is gone") + } + } catch { + case e: Exception => logError("Exception in statusUpdate", e) + } + } + // Update the DAGScheduler without holding a lock on this, since that can deadlock + if (failedExecutor != None) { + dagScheduler.executorLost(failedExecutor.get) + backend.reviveOffers() + } + } + + def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long) { + taskSetManager.handleTaskGettingResult(tid) + } + + def handleSuccessfulTask( + taskSetManager: TaskSetManager, + tid: Long, + taskResult: DirectTaskResult[_]) = synchronized { + taskSetManager.handleSuccessfulTask(tid, taskResult) + } + + def handleFailedTask( + taskSetManager: TaskSetManager, + tid: Long, + taskState: TaskState, + reason: Option[TaskEndReason]) = synchronized { + taskSetManager.handleFailedTask(tid, taskState, reason) + if (taskState != TaskState.KILLED) { + // Need to revive offers again now that the task set manager state has been updated to + // reflect failed tasks that need to be re-run. + backend.reviveOffers() + } + } + + def error(message: String) { + synchronized { + if (activeTaskSets.size > 0) { + // Have each task set throw a SparkException with the error + for ((taskSetId, manager) <- activeTaskSets) { + try { + manager.error(message) + } catch { + case e: Exception => logError("Exception in error callback", e) + } + } + } else { + // No task sets are active but we still got an error. Just exit since this + // must mean the error is during registration. + // It might be good to do something smarter here in the future. + logError("Exiting due to error from cluster scheduler: " + message) + System.exit(1) + } + } + } + + override def stop() { + if (backend != null) { + backend.stop() + } + if (taskResultGetter != null) { + taskResultGetter.stop() + } + + // sleeping for an arbitrary 5 seconds : to ensure that messages are sent out. + // TODO: Do something better ! + Thread.sleep(5000L) + } + + override def defaultParallelism() = backend.defaultParallelism() + + // Check for speculatable tasks in all our active jobs. + def checkSpeculatableTasks() { + var shouldRevive = false + synchronized { + shouldRevive = rootPool.checkSpeculatableTasks() + } + if (shouldRevive) { + backend.reviveOffers() + } + } + + // Check for pending tasks in all our active jobs. + def hasPendingTasks: Boolean = { + synchronized { + rootPool.hasPendingTasks() + } + } + + def executorLost(executorId: String, reason: ExecutorLossReason) { + var failedExecutor: Option[String] = None + + synchronized { + if (activeExecutorIds.contains(executorId)) { + val hostPort = executorIdToHost(executorId) + logError("Lost executor %s on %s: %s".format(executorId, hostPort, reason)) + removeExecutor(executorId) + failedExecutor = Some(executorId) + } else { + // We may get multiple executorLost() calls with different loss reasons. For example, one + // may be triggered by a dropped connection from the slave while another may be a report + // of executor termination from Mesos. We produce log messages for both so we eventually + // report the termination reason. + logError("Lost an executor " + executorId + " (already removed): " + reason) + } + } + // Call dagScheduler.executorLost without holding the lock on this to prevent deadlock + if (failedExecutor != None) { + dagScheduler.executorLost(failedExecutor.get) + backend.reviveOffers() + } + } + + /** Remove an executor from all our data structures and mark it as lost */ + private def removeExecutor(executorId: String) { + activeExecutorIds -= executorId + val host = executorIdToHost(executorId) + val execs = executorsByHost.getOrElse(host, new HashSet) + execs -= executorId + if (execs.isEmpty) { + executorsByHost -= host + } + executorIdToHost -= executorId + rootPool.executorLost(executorId, host) + } + + def executorGained(execId: String, host: String) { + dagScheduler.executorGained(execId, host) + } + + def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized { + executorsByHost.get(host).map(_.toSet) + } + + def hasExecutorsAliveOnHost(host: String): Boolean = synchronized { + executorsByHost.contains(host) + } + + def isExecutorAlive(execId: String): Boolean = synchronized { + activeExecutorIds.contains(execId) + } + + // By default, rack is unknown + def getRackForHost(value: String): Option[String] = None +} + + +private[spark] object TaskSchedulerImpl { + /** + * 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 + + // order keyList based on population of value in map + val keyList = _keyList.sortWith( + (left, right) => map(left).size > map(right).size + ) + + val retval = new ArrayBuffer[T](keyList.size * 2) + var index = 0 + var found = true + + while (found) { + found = false + for (key <- keyList) { + val containerList: ArrayBuffer[T] = map.get(key).getOrElse(null) + assert(containerList != null) + // Get the index'th entry for this host - if present + if (index < containerList.size){ + retval += containerList.apply(index) + found = true + } + } + index += 1 + } + + retval.toList + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 0fe413a7c4..0ac982909c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -48,7 +48,7 @@ import java.io.NotSerializableException * task set will be aborted */ private[spark] class TaskSetManager( - sched: ClusterScheduler, + sched: TaskSchedulerImpl, val taskSet: TaskSet, val maxTaskFailures: Int, clock: Clock = SystemClock) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 5797783793..5c534a6f43 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -29,7 +29,7 @@ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import org.apache.spark.{SparkException, Logging, TaskState} import org.apache.spark.{Logging, SparkException, TaskState} -import org.apache.spark.scheduler.{ClusterScheduler, SchedulerBackend, SlaveLost, TaskDescription, +import org.apache.spark.scheduler.{TaskSchedulerImpl, SchedulerBackend, SlaveLost, TaskDescription, WorkerOffer} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -43,7 +43,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} * (spark.deploy.*). */ private[spark] -class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: ActorSystem) +class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: ActorSystem) extends SchedulerBackend with Logging { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index 2fbd725d75..ec3e68e970 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -21,10 +21,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.scheduler.ClusterScheduler +import org.apache.spark.scheduler.TaskSchedulerImpl private[spark] class SimrSchedulerBackend( - scheduler: ClusterScheduler, + scheduler: TaskSchedulerImpl, sc: SparkContext, driverFilePath: String) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) 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 1d38f0d956..404ce7a452 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 @@ -22,11 +22,11 @@ import scala.collection.mutable.HashMap import org.apache.spark.{Logging, SparkContext} import org.apache.spark.deploy.client.{Client, ClientListener} import org.apache.spark.deploy.{Command, ApplicationDescription} -import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, ClusterScheduler} +import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} import org.apache.spark.util.Utils private[spark] class SparkDeploySchedulerBackend( - scheduler: ClusterScheduler, + scheduler: TaskSchedulerImpl, sc: SparkContext, masters: Array[String], appName: String) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 5481828111..39573fc8c9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -30,7 +30,7 @@ import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} import org.apache.spark.{SparkException, Logging, SparkContext, TaskState} -import org.apache.spark.scheduler.ClusterScheduler +import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend /** @@ -44,7 +44,7 @@ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend * remove this. */ private[spark] class CoarseMesosSchedulerBackend( - scheduler: ClusterScheduler, + scheduler: TaskSchedulerImpl, sc: SparkContext, master: String, appName: String) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 773b980c53..6aa788c460 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -31,7 +31,7 @@ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTas import org.apache.spark.{Logging, SparkException, SparkContext, TaskState} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SchedulerBackend, SlaveLost, - TaskDescription, ClusterScheduler, WorkerOffer} + TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.util.Utils /** @@ -40,7 +40,7 @@ import org.apache.spark.util.Utils * from multiple apps can run on different cores) and in time (a core can switch ownership). */ private[spark] class MesosSchedulerBackend( - scheduler: ClusterScheduler, + scheduler: TaskSchedulerImpl, sc: SparkContext, master: String, appName: String) diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 6b5f1a5dc2..69c1c04843 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -24,7 +24,7 @@ import akka.actor.{Actor, ActorRef, Props} import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} -import org.apache.spark.scheduler.{SchedulerBackend, ClusterScheduler, WorkerOffer} +import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} private case class ReviveOffers() @@ -38,7 +38,7 @@ private case class KillTask(taskId: Long) * and the ClusterScheduler. */ private[spark] class LocalActor( - scheduler: ClusterScheduler, + scheduler: TaskSchedulerImpl, executorBackend: LocalBackend, private val totalCores: Int) extends Actor with Logging { @@ -78,7 +78,7 @@ private[spark] class LocalActor( * master all run in the same JVM. It sits behind a ClusterScheduler and handles launching tasks * on a single Executor (created by the LocalBackend) running locally. */ -private[spark] class LocalBackend(scheduler: ClusterScheduler, val totalCores: Int) +private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: Int) extends SchedulerBackend with ExecutorBackend { var localActor: ActorRef = null diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index d4a7a11515..9deed568ac 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark import org.scalatest.{FunSuite, PrivateMethodTester} -import org.apache.spark.scheduler.{ClusterScheduler, TaskScheduler} +import org.apache.spark.scheduler.{TaskSchedulerImpl, TaskScheduler} import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend @@ -27,13 +27,13 @@ import org.apache.spark.scheduler.local.LocalBackend class SparkContextSchedulerCreationSuite extends FunSuite with PrivateMethodTester with LocalSparkContext with Logging { - def createTaskScheduler(master: String): ClusterScheduler = { + def createTaskScheduler(master: String): TaskSchedulerImpl = { // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. sc = new SparkContext("local", "test") val createTaskSchedulerMethod = PrivateMethod[TaskScheduler]('createTaskScheduler) val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master, "test") - sched.asInstanceOf[ClusterScheduler] + sched.asInstanceOf[TaskSchedulerImpl] } test("bad-master") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala index 35a06c4875..702edb862f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala @@ -29,7 +29,7 @@ class FakeTaskSetManager( initPriority: Int, initStageId: Int, initNumTasks: Int, - clusterScheduler: ClusterScheduler, + clusterScheduler: TaskSchedulerImpl, taskSet: TaskSet) extends TaskSetManager(clusterScheduler, taskSet, 0) { @@ -104,7 +104,7 @@ class FakeTaskSetManager( class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging { - def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: ClusterScheduler, taskSet: TaskSet): FakeTaskSetManager = { + def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: TaskSchedulerImpl, taskSet: TaskSet): FakeTaskSetManager = { new FakeTaskSetManager(priority, stage, numTasks, cs , taskSet) } @@ -131,7 +131,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging test("FIFO Scheduler Test") { sc = new SparkContext("local", "ClusterSchedulerSuite") - val clusterScheduler = new ClusterScheduler(sc) + val clusterScheduler = new TaskSchedulerImpl(sc) var tasks = ArrayBuffer[Task[_]]() val task = new FakeTask(0) tasks += task @@ -158,7 +158,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging test("Fair Scheduler Test") { sc = new SparkContext("local", "ClusterSchedulerSuite") - val clusterScheduler = new ClusterScheduler(sc) + val clusterScheduler = new TaskSchedulerImpl(sc) var tasks = ArrayBuffer[Task[_]]() val task = new FakeTask(0) tasks += task @@ -215,7 +215,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging test("Nested Pool Test") { sc = new SparkContext("local", "ClusterSchedulerSuite") - val clusterScheduler = new ClusterScheduler(sc) + val clusterScheduler = new TaskSchedulerImpl(sc) var tasks = ArrayBuffer[Task[_]]() val task = new FakeTask(0) tasks += task diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 9784920653..2265619570 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.storage.TaskResultBlockId * Used to test the case where a BlockManager evicts the task result (or dies) before the * TaskResult is retrieved. */ -class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler) +class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedulerImpl) extends TaskResultGetter(sparkEnv, scheduler) { var removedResult = false @@ -92,8 +92,8 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA sc = new SparkContext("local[1,1]", "test") // If this test hangs, it's probably because no resource offers were made after the task // failed. - val scheduler: ClusterScheduler = sc.taskScheduler match { - case clusterScheduler: ClusterScheduler => + val scheduler: TaskSchedulerImpl = sc.taskScheduler match { + case clusterScheduler: TaskSchedulerImpl => clusterScheduler case _ => assert(false, "Expect local cluster to use ClusterScheduler") diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index b34b6f32f2..771a64ff6c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -58,7 +58,7 @@ class FakeDAGScheduler(taskScheduler: FakeClusterScheduler) extends DAGScheduler * to work, and these are required for locality in TaskSetManager. */ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* execId, host */) - extends ClusterScheduler(sc) + extends TaskSchedulerImpl(sc) { val startedTasks = new ArrayBuffer[Long] val endedTasks = new mutable.HashMap[Long, TaskEndReason] -- cgit v1.2.3 From b8ae096a40eb0f83aac889deb061a9484effd9aa Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Sat, 21 Dec 2013 23:28:48 -0800 Subject: Fix build error in test --- .../src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 771a64ff6c..3dcb01ae5e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -283,7 +283,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted // after the last failure. - (1 to manager.MAX_TASK_FAILURES).foreach { index => + (1 to manager.maxTaskFailures).foreach { index => val offerResult = manager.resourceOffer("exec1", "host1", 1, ANY) assert(offerResult != None, "Expect resource offer on iteration %s to return a task".format(index)) -- cgit v1.2.3 From c979eecdf6a11462595aba9d5b8fc942682cf85d Mon Sep 17 00:00:00 2001 From: "wangda.tan" Date: Sun, 22 Dec 2013 21:43:15 +0800 Subject: added changes according to comments from rxin --- .../org/apache/spark/ui/exec/ExecutorsUI.scala | 24 +++++++-------------- .../org/apache/spark/ui/jobs/ExecutorSummary.scala | 5 +++-- .../org/apache/spark/ui/jobs/ExecutorTable.scala | 4 ++-- .../scala/org/apache/spark/ui/jobs/IndexPage.scala | 4 ---- .../apache/spark/ui/jobs/JobProgressListener.scala | 25 +++++++--------------- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 4 ++-- .../org/apache/spark/ui/jobs/StageTable.scala | 2 +- 7 files changed, 24 insertions(+), 44 deletions(-) (limited to 'core/src') 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 f62ae37466..a31a7e1d58 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 @@ -56,7 +56,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { 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", "Duration", "Shuffle Read", + "Active tasks", "Failed tasks", "Complete tasks", "Total tasks", "Task Time", "Shuffle Read", "Shuffle Write") def execRow(kv: Seq[String]) = { @@ -169,21 +169,13 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { // update shuffle read/write if (null != taskEnd.taskMetrics) { - val shuffleRead = taskEnd.taskMetrics.shuffleReadMetrics - shuffleRead match { - case Some(s) => - val newShuffleRead = executorToShuffleRead.getOrElse(eid, 0L) + s.remoteBytesRead - executorToShuffleRead.put(eid, newShuffleRead) - case _ => {} - } - val shuffleWrite = taskEnd.taskMetrics.shuffleWriteMetrics - shuffleWrite match { - case Some(s) => { - val newShuffleWrite = executorToShuffleWrite.getOrElse(eid, 0L) + s.shuffleBytesWritten - executorToShuffleWrite.put(eid, newShuffleWrite) - } - case _ => {} - } + taskEnd.taskMetrics.shuffleReadMetrics.foreach(shuffleRead => + executorToShuffleRead.put(eid, executorToShuffleRead.getOrElse(eid, 0L) + + shuffleRead.remoteBytesRead)) + + taskEnd.taskMetrics.shuffleWriteMetrics.foreach(shuffleWrite => + executorToShuffleWrite.put(eid, executorToShuffleWrite.getOrElse(eid, 0L) + + shuffleWrite.shuffleBytesWritten)) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala index 75c0dd2c7f..3c53e88380 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala @@ -17,8 +17,9 @@ package org.apache.spark.ui.jobs -private[spark] class ExecutorSummary() { - var duration : Long = 0 +/** class for reporting aggregated metrics for each executors in stageUI */ +private[spark] class ExecutorSummary { + var taskTime : Long = 0 var failedTasks : Int = 0 var succeededTasks : Int = 0 var shuffleRead : Long = 0 diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 763d5a344b..0e9dd4a8c7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -40,7 +40,7 @@ private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) - + @@ -61,7 +61,7 @@ private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) case (k,v) => { - + diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 854afb665a..ca5a28625b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -56,10 +56,6 @@ private[spark] class IndexPage(parent: JobProgressUI) { {parent.formatDuration(now - listener.sc.startTime)}
  • Scheduling Mode: {parent.sc.getSchedulingMode}
  • -
  • - Executor Summary: - {listener.stageIdToExecutorSummaries.size} -
  • Active Stages: {activeStages.size} 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 64ce715993..07a42f0503 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 @@ -144,23 +144,14 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList } // update duration - y.duration += taskEnd.taskInfo.duration - - // update shuffle read/write - if (null != taskEnd.taskMetrics) { - val shuffleRead = taskEnd.taskMetrics.shuffleReadMetrics - shuffleRead match { - case Some(s) => - y.shuffleRead += s.remoteBytesRead - case _ => {} - } - val shuffleWrite = taskEnd.taskMetrics.shuffleWriteMetrics - shuffleWrite match { - case Some(s) => { - y.shuffleWrite += s.shuffleBytesWritten - } - case _ => {} - } + y.taskTime += taskEnd.taskInfo.duration + + taskEnd.taskMetrics.shuffleReadMetrics.foreach { shuffleRead => + y.shuffleRead += shuffleRead.remoteBytesRead + } + + taskEnd.taskMetrics.shuffleWriteMetrics.foreach { shuffleWrite => + y.shuffleWrite += shuffleWrite.shuffleBytesWritten } } case _ => {} 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 c077613b1d..d8a6c9e2dc 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 @@ -66,7 +66,7 @@ private[spark] class StagePage(parent: JobProgressUI) {
    • - Total duration across all tasks: + Total task time across all tasks: {parent.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)}
    • {if (hasShuffleRead) @@ -163,9 +163,9 @@ private[spark] class StagePage(parent: JobProgressUI) { val executorTable = new ExecutorTable(parent, stageId) val content = summary ++ -

      Summary Metrics for Executors

      ++ executorTable.toNodeSeq() ++

      Summary Metrics for {numCompleted} Completed Tasks

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

      Aggregated Metrics by Executors

      ++ executorTable.toNodeSeq() ++

      Tasks

      ++ taskTable headerSparkPage(content, parent.sc, "Details for Stage %d".format(stageId), Stages) 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 9ad6de3c6d..463d85dfd5 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 @@ -48,7 +48,7 @@ private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgr {if (isFairScheduler) {
  • } else {}} - + -- cgit v1.2.3 From b7bfae1afecad0ae79d5d040d2e02e390c272efb Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Sun, 22 Dec 2013 07:27:28 -0800 Subject: Correctly merged in maxTaskFailures fix --- core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 2 +- core/src/test/scala/org/apache/spark/FailureSuite.scala | 4 ++-- .../scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala | 2 +- .../test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 0ac982909c..aa3fb0b35a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -540,7 +540,7 @@ private[spark] class TaskSetManager( if (numFailures(index) >= maxTaskFailures) { logError("Task %s:%d failed %d times; aborting job".format( taskSet.id, index, maxTaskFailures)) - abort("Task %s:%d failed more than %d times (most recent failure: %s)".format( + abort("Task %s:%d failed %d times (most recent failure: %s)".format( taskSet.id, index, maxTaskFailures, failureReason)) } } diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index af448fcb37..befdc1589f 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -42,7 +42,7 @@ 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") { - sc = new SparkContext("local[1,1]", "test") + sc = new SparkContext("local[1,2]", "test") val results = sc.makeRDD(1 to 3, 3).map { x => FailureSuiteState.synchronized { FailureSuiteState.tasksRun += 1 @@ -62,7 +62,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { // Run a map-reduce job in which a reduce task deterministically fails once. test("failure in a two-stage job") { - sc = new SparkContext("local[1,1]", "test") + sc = new SparkContext("local[1,2]", "test") val results = sc.makeRDD(1 to 3).map(x => (x, x)).groupByKey(3).map { case (k, v) => FailureSuiteState.synchronized { diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 9deed568ac..f28d5c7b13 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -53,7 +53,7 @@ class SparkContextSchedulerCreationSuite test("local-n") { val sched = createTaskScheduler("local[5]") - assert(sched.maxTaskFailures === 0) + assert(sched.maxTaskFailures === 1) sched.backend match { case s: LocalBackend => assert(s.totalCores === 5) case _ => fail() diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 2265619570..ca97f7d2a5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -89,7 +89,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA test("task retried if result missing from block manager") { // Set the maximum number of task failures to > 0, so that the task set isn't aborted // after the result is missing. - sc = new SparkContext("local[1,1]", "test") + sc = new SparkContext("local[1,2]", "test") // If this test hangs, it's probably because no resource offers were made after the task // failed. val scheduler: TaskSchedulerImpl = sc.taskScheduler match { -- cgit v1.2.3 From 2f689ba97b437092bf52063cface12aa9ee09bf3 Mon Sep 17 00:00:00 2001 From: "wangda.tan" Date: Mon, 23 Dec 2013 15:03:45 +0800 Subject: SPARK-968, added executor address showing in aggregated metrics by executors table --- .../main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 0e9dd4a8c7..0dd876480a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -21,6 +21,7 @@ import scala.xml.Node import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.util.Utils +import scala.collection.mutable /** Page showing executor summary */ private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) { @@ -40,6 +41,7 @@ private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int)
    Executor IDDurationTask Time Total Tasks Failed Tasks Succeeded Tasks
    {k}{parent.formatDuration(v.duration)}{parent.formatDuration(v.taskTime)} {v.failedTasks + v.succeededTasks} {v.failedTasks} {v.succeededTasks} Pool NameDescription SubmittedDurationTask Time Tasks: Succeeded/Total Shuffle Read Shuffle Write
    + @@ -54,6 +56,16 @@ private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) } private def createExecutorTable() : Seq[Node] = { + // make a executor-id -> address map + val executorIdToAddress = mutable.HashMap[String, String]() + val storageStatusList = parent.sc.getExecutorStorageStatus + for (statusId <- 0 until storageStatusList.size) { + val blockManagerId = parent.sc.getExecutorStorageStatus(statusId).blockManagerId + val address = blockManagerId.hostPort + val executorId = blockManagerId.executorId + executorIdToAddress.put(executorId, address) + } + val executorIdToSummary = listener.stageIdToExecutorSummaries.get(stageId) executorIdToSummary match { case Some(x) => { @@ -61,6 +73,7 @@ private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) case (k,v) => { + -- cgit v1.2.3 From 2573add94cf920a88f74d80d8ea94218d812704d Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 24 Dec 2013 18:30:31 +0530 Subject: spark-544, introducing SparkConf and related configuration overhaul. --- .../scala/org/apache/spark/MapOutputTracker.scala | 7 +- .../main/scala/org/apache/spark/Partitioner.scala | 4 +- .../main/scala/org/apache/spark/SparkConf.scala | 71 +++++++++++ .../main/scala/org/apache/spark/SparkContext.scala | 140 +++++++++++---------- .../src/main/scala/org/apache/spark/SparkEnv.scala | 44 ++++--- .../org/apache/spark/api/python/PythonRDD.scala | 6 +- .../org/apache/spark/broadcast/Broadcast.scala | 6 +- .../apache/spark/broadcast/BroadcastFactory.scala | 4 +- .../org/apache/spark/broadcast/HttpBroadcast.scala | 14 +-- .../apache/spark/broadcast/TorrentBroadcast.scala | 9 +- .../spark/deploy/ApplicationDescription.scala | 2 +- .../org/apache/spark/deploy/SparkHadoopUtil.scala | 3 +- .../org/apache/spark/deploy/client/Client.scala | 13 +- .../apache/spark/deploy/client/TestClient.scala | 7 +- .../org/apache/spark/deploy/master/Master.scala | 31 ++--- .../spark/deploy/master/MasterArguments.scala | 7 +- .../deploy/master/SparkZooKeeperSession.scala | 7 +- .../master/ZooKeeperLeaderElectionAgent.scala | 9 +- .../deploy/master/ZooKeeperPersistenceEngine.scala | 8 +- .../spark/deploy/master/ui/MasterWebUI.scala | 2 +- .../org/apache/spark/deploy/worker/Worker.scala | 28 ++--- .../spark/deploy/worker/ui/WorkerWebUI.scala | 6 +- .../executor/CoarseGrainedExecutorBackend.scala | 6 +- .../scala/org/apache/spark/executor/Executor.scala | 10 +- .../org/apache/spark/io/CompressionCodec.scala | 14 ++- .../org/apache/spark/metrics/MetricsSystem.scala | 10 +- .../apache/spark/network/ConnectionManager.scala | 24 ++-- .../org/apache/spark/network/ReceiverTest.scala | 4 +- .../org/apache/spark/network/SenderTest.scala | 4 +- .../apache/spark/network/netty/ShuffleCopier.scala | 8 +- .../scala/org/apache/spark/rdd/CheckpointRDD.scala | 6 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 1 + .../org/apache/spark/scheduler/JobLogger.scala | 2 +- .../spark/scheduler/SchedulableBuilder.scala | 6 +- .../spark/scheduler/cluster/ClusterScheduler.scala | 11 +- .../scheduler/cluster/ClusterTaskSetManager.scala | 19 +-- .../cluster/CoarseGrainedSchedulerBackend.scala | 12 +- .../scheduler/cluster/SimrSchedulerBackend.scala | 4 +- .../cluster/SparkDeploySchedulerBackend.scala | 6 +- .../spark/scheduler/cluster/TaskResultGetter.scala | 3 +- .../mesos/CoarseMesosSchedulerBackend.scala | 10 +- .../cluster/mesos/MesosSchedulerBackend.scala | 4 +- .../spark/scheduler/local/LocalScheduler.scala | 5 +- .../apache/spark/serializer/KryoSerializer.scala | 10 +- .../spark/storage/BlockFetcherIterator.scala | 4 +- .../org/apache/spark/storage/BlockManager.scala | 38 +++--- .../apache/spark/storage/BlockManagerMaster.scala | 11 +- .../spark/storage/BlockManagerMasterActor.scala | 10 +- .../apache/spark/storage/BlockObjectWriter.scala | 5 +- .../apache/spark/storage/DiskBlockManager.scala | 2 +- .../apache/spark/storage/ShuffleBlockManager.scala | 7 +- .../org/apache/spark/storage/ThreadingTest.scala | 6 +- .../main/scala/org/apache/spark/ui/SparkUI.scala | 2 +- .../org/apache/spark/ui/UIWorkloadGenerator.scala | 4 +- .../org/apache/spark/ui/env/EnvironmentUI.scala | 2 +- .../apache/spark/ui/jobs/JobProgressListener.scala | 2 +- .../scala/org/apache/spark/util/AkkaUtils.scala | 25 ++-- .../org/apache/spark/util/MetadataCleaner.scala | 12 +- .../org/apache/spark/util/SizeEstimator.scala | 7 +- .../main/scala/org/apache/spark/util/Utils.scala | 7 +- .../org/apache/spark/MapOutputTrackerSuite.scala | 16 +-- .../apache/spark/metrics/MetricsSystemSuite.scala | 8 +- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 23 ++-- .../apache/spark/scheduler/JobLoggerSuite.scala | 2 +- .../scheduler/cluster/ClusterSchedulerSuite.scala | 2 +- .../cluster/ClusterTaskSetManagerSuite.scala | 4 +- .../scheduler/cluster/TaskResultGetterSuite.scala | 2 +- .../apache/spark/storage/BlockManagerSuite.scala | 95 +++++++------- .../spark/storage/DiskBlockManagerSuite.scala | 12 +- .../org/apache/spark/util/SizeEstimatorSuite.scala | 4 +- .../spark/examples/bagel/WikipediaPageRank.scala | 10 +- .../bagel/WikipediaPageRankStandalone.scala | 8 +- .../spark/streaming/examples/ActorWordCount.scala | 3 +- .../apache/spark/mllib/recommendation/ALS.scala | 13 +- .../spark/deploy/yarn/ApplicationMaster.scala | 16 +-- .../org/apache/spark/deploy/yarn/Client.scala | 4 +- .../apache/spark/deploy/yarn/ClientArguments.scala | 2 +- .../apache/spark/deploy/yarn/WorkerLauncher.scala | 4 +- .../spark/deploy/yarn/YarnAllocationHandler.scala | 4 +- .../cluster/YarnClientSchedulerBackend.scala | 4 +- project/SparkBuild.scala | 3 +- .../scala/org/apache/spark/repl/SparkILoop.scala | 7 +- .../scala/org/apache/spark/repl/SparkIMain.scala | 7 +- .../org/apache/spark/streaming/Checkpoint.scala | 3 +- .../org/apache/spark/streaming/Scheduler.scala | 6 +- .../apache/spark/streaming/StreamingContext.scala | 2 +- .../streaming/dstream/NetworkInputDStream.scala | 6 +- .../apache/spark/streaming/CheckpointSuite.scala | 6 +- .../apache/spark/streaming/InputStreamsSuite.scala | 18 +-- .../org/apache/spark/streaming/TestSuiteBase.scala | 11 +- .../spark/deploy/yarn/ApplicationMaster.scala | 16 +-- .../org/apache/spark/deploy/yarn/Client.scala | 6 +- .../apache/spark/deploy/yarn/ClientArguments.scala | 2 +- .../apache/spark/deploy/yarn/WorkerLauncher.scala | 4 +- .../spark/deploy/yarn/YarnAllocationHandler.scala | 2 +- .../cluster/YarnClientSchedulerBackend.scala | 4 +- 96 files changed, 612 insertions(+), 478 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/SparkConf.scala (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index ccffcc356c..4520edb10d 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -50,9 +50,9 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster } } -private[spark] class MapOutputTracker extends Logging { +private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { - private val timeout = AkkaUtils.askTimeout + private val timeout = AkkaUtils.askTimeout(conf) // Set to the MapOutputTrackerActor living on the driver var trackerActor: Either[ActorRef, ActorSelection] = _ @@ -192,7 +192,8 @@ private[spark] class MapOutputTracker extends Logging { } } -private[spark] class MapOutputTrackerMaster extends MapOutputTracker { +private[spark] class MapOutputTrackerMaster(conf: SparkConf) + extends MapOutputTracker(conf) { // Cache a serialized version of the output statuses for each shuffle to send them out faster private var cacheEpoch = epoch diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index bcec41c439..04c1eedfeb 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -32,6 +32,8 @@ abstract class Partitioner extends Serializable { } object Partitioner { + + import SparkContext.{globalConf => conf} /** * Choose a partitioner to use for a cogroup-like operation between a number of RDDs. * @@ -52,7 +54,7 @@ object Partitioner { for (r <- bySize if r.partitioner != None) { return r.partitioner.get } - if (System.getProperty("spark.default.parallelism") != null) { + if (conf.getOrElse("spark.default.parallelism", null) != null) { return new HashPartitioner(rdd.context.defaultParallelism) } else { return new HashPartitioner(bySize.head.partitions.size) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala new file mode 100644 index 0000000000..9a4eefad2e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -0,0 +1,71 @@ +package org.apache.spark + +import scala.collection.JavaConversions._ +import scala.collection.concurrent.TrieMap + +import com.typesafe.config.ConfigFactory + +private[spark] class SparkConf(loadClasspathRes: Boolean = true) extends Serializable { + @transient lazy val config = ConfigFactory.systemProperties() + .withFallback(ConfigFactory.parseResources("spark.conf")) + // TODO this should actually be synchronized + private val configMap = TrieMap[String, String]() + + if (loadClasspathRes && !config.entrySet().isEmpty) { + for (e <- config.entrySet()) { + configMap += ((e.getKey, e.getValue.unwrapped().toString)) + } + } + + def setMasterUrl(master: String) = { + if (master != null) + configMap += (("spark.master", master)) + this + } + + def setAppName(name: String) = { + if (name != null) + configMap += (("spark.appName", name)) + this + } + + def setJars(jars: Seq[String]) = { + if (!jars.isEmpty) + configMap += (("spark.jars", jars.mkString(","))) + this + } + + def set(k: String, value: String) = { + configMap += ((k, value)) + this + } + + def setSparkHome(home: String) = { + if (home != null) + configMap += (("spark.home", home)) + this + } + + def set(map: Seq[(String, String)]) = { + if (map != null && !map.isEmpty) + configMap ++= map + this + } + + def get(k: String): String = { + configMap(k) + } + + def getAllConfiguration = configMap.clone.entrySet().iterator + + def getOrElse(k: String, defaultValue: String): String = { + configMap.getOrElse(k, defaultValue) + } + + override def clone: SparkConf = { + val conf = new SparkConf(false) + conf.set(configMap.toSeq) + conf + } + +} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a0f794edfd..4300b07bdb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -22,91 +22,99 @@ import java.net.URI import java.util.Properties import java.util.concurrent.atomic.AtomicInteger -import scala.collection.Map +import scala.collection.{Map, immutable} +import scala.collection.JavaConversions._ import scala.collection.generic.Growable -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap + +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.reflect.{ClassTag, classTag} +import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.ArrayWritable -import org.apache.hadoop.io.BooleanWritable -import org.apache.hadoop.io.BytesWritable -import org.apache.hadoop.io.DoubleWritable -import org.apache.hadoop.io.FloatWritable -import org.apache.hadoop.io.IntWritable -import org.apache.hadoop.io.LongWritable -import org.apache.hadoop.io.NullWritable -import org.apache.hadoop.io.Text -import org.apache.hadoop.io.Writable -import org.apache.hadoop.mapred.FileInputFormat -import org.apache.hadoop.mapred.InputFormat -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.SequenceFileInputFormat -import org.apache.hadoop.mapred.TextInputFormat -import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import org.apache.hadoop.mapreduce.{Job => NewHadoopJob} +import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, +FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} +import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat, +TextInputFormat} +import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} - import org.apache.mesos.MesosNativeLibrary import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, - SparkDeploySchedulerBackend, ClusterScheduler, SimrSchedulerBackend} -import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} +import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend, +SimrSchedulerBackend, SparkDeploySchedulerBackend} +import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, +MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalScheduler -import org.apache.spark.scheduler.StageInfo import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, - TimeStampedHashMap, Utils} +import org.apache.spark.util._ /** * 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. * - * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). - * @param appName A name for your application, to display on the cluster web UI. - * @param sparkHome Location where Spark is installed on cluster nodes. - * @param jars Collection of JARs to send to the cluster. These can be paths on the local file - * system or HDFS, HTTP, HTTPS, or FTP URLs. + * @param conf a Spark Config object describing the context configuration. Any settings in this + * config overrides the default configs as well as system properties. + * * @param environment Environment variables to set on worker nodes. */ class SparkContext( - val master: String, - val appName: String, - val sparkHome: String = null, - val jars: Seq[String] = Nil, + val conf: SparkConf, val environment: Map[String, String] = Map(), // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, etc) // too. This is typically generated from InputFormatInfo.computePreferredLocations .. host, set // of data-local splits on host - val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = - scala.collection.immutable.Map()) + val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = immutable.Map()) extends Logging { - // Ensure logging is initialized before we spawn any threads - initLogging() + /** + * 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. + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + * @param sparkHome Location where Spark is installed on cluster nodes. + * @param jars Collection of JARs to send to the cluster. These can be paths on the local file + * system or HDFS, HTTP, HTTPS, or FTP URLs. + * @param environment Environment variables to set on worker nodes. + */ + def this(master: String, appName: String, sparkHome: String = null, + jars: Seq[String] = Nil, environment: Map[String, String] = Map(), + preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = + immutable.Map()) = + this(new SparkConf(false).setAppName(appName).setMasterUrl(master) + .setJars(jars).set(environment.toSeq).setSparkHome(sparkHome), + environment, preferredNodeLocationData) // Set Spark driver host and port system properties - if (System.getProperty("spark.driver.host") == null) { - System.setProperty("spark.driver.host", Utils.localHostName()) - } - if (System.getProperty("spark.driver.port") == null) { - System.setProperty("spark.driver.port", "0") - } + Try(conf.get("spark.driver.host")) + .getOrElse(conf.set("spark.driver.host", Utils.localHostName())) + + Try(conf.get("spark.driver.port")) + .getOrElse(conf.set("spark.driver.port", "0")) + + val jars: Seq[String] = if (conf.getOrElse("spark.jars", null) != null) { + conf.get("spark.jars").split(",") + } else null + + val master = conf.get("spark.master") + val appName = conf.get("spark.appName") val isLocal = (master == "local" || master.startsWith("local[")) + // Ensure logging is initialized before we spawn any threads + initLogging() + // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.createFromSystemProperties( "", - System.getProperty("spark.driver.host"), - System.getProperty("spark.driver.port").toInt, + conf.get("spark.driver.host"), + conf.get("spark.driver.port").toInt, + conf, true, isLocal) SparkEnv.set(env) @@ -165,24 +173,24 @@ class SparkContext( /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { val env = SparkEnv.get - val conf = SparkHadoopUtil.get.newConfiguration() + val hadoopConf = SparkHadoopUtil.get.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")) - conf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) - conf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) - conf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) + hadoopConf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) + hadoopConf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) + hadoopConf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) + hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) } // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar" Utils.getSystemProperties.foreach { case (key, value) => if (key.startsWith("spark.hadoop.")) { - conf.set(key.substring("spark.hadoop.".length), value) + hadoopConf.set(key.substring("spark.hadoop.".length), value) } } - val bufferSize = System.getProperty("spark.buffer.size", "65536") - conf.set("io.file.buffer.size", bufferSize) - conf + val bufferSize = conf.getOrElse("spark.buffer.size", "65536") + hadoopConf.set("io.file.buffer.size", bufferSize) + hadoopConf } private[spark] var checkpointDir: Option[String] = None @@ -695,10 +703,8 @@ class SparkContext( * (in that order of preference). If neither of these is set, return None. */ private[spark] def getSparkHome(): Option[String] = { - if (sparkHome != null) { - Some(sparkHome) - } else if (System.getProperty("spark.home") != null) { - Some(System.getProperty("spark.home")) + if (conf.getOrElse("spark.home", null) != null) { + Some(conf.get("spark.home")) } else if (System.getenv("SPARK_HOME") != null) { Some(System.getenv("SPARK_HOME")) } else { @@ -909,6 +915,14 @@ object SparkContext { private[spark] val SPARK_UNKNOWN_USER = "" + private lazy val conf = new SparkConf() + + private[spark] def globalConf = { + if (SparkEnv.get != null) { + SparkEnv.get.conf + } else conf + } + implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double) = 0.0 @@ -1020,7 +1034,7 @@ object SparkContext { /** 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")) + Try(globalConf.get("spark.executor.memory")).toOption .orElse(Option(System.getenv("SPARK_MEM"))) .map(Utils.memoryStringToMb) .getOrElse(512) @@ -1123,7 +1137,7 @@ object SparkContext { case mesosUrl @ MESOS_REGEX(_) => MesosNativeLibrary.load() val scheduler = new ClusterScheduler(sc) - val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean + val coarseGrained = globalConf.getOrElse("spark.mesos.coarse", "false").toBoolean val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { new CoarseMesosSchedulerBackend(scheduler, sc, url, appName) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 826f5c2d8c..78e4ae27b2 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -54,7 +54,8 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, - val metricsSystem: MetricsSystem) { + val metricsSystem: MetricsSystem, + val conf: SparkConf) { private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() @@ -114,25 +115,27 @@ object SparkEnv extends Logging { executorId: String, hostname: String, port: Int, + conf: SparkConf, isDriver: Boolean, isLocal: Boolean): SparkEnv = { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port, + conf = conf) // Bit of a hack: If this is the driver and our port was 0 (meaning bind to any free port), // figure out which port number Akka actually bound to and set spark.driver.port to it. if (isDriver && port == 0) { - System.setProperty("spark.driver.port", boundPort.toString) + conf.set("spark.driver.port", boundPort.toString) } // set only if unset until now. - if (System.getProperty("spark.hostPort", null) == null) { + if (conf.getOrElse("spark.hostPort", null) == null) { if (!isDriver){ // unexpected Utils.logErrorWithStack("Unexpected NOT to have spark.hostPort set") } Utils.checkHost(hostname) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + conf.set("spark.hostPort", hostname + ":" + boundPort) } val classLoader = Thread.currentThread.getContextClassLoader @@ -140,25 +143,25 @@ object SparkEnv extends Logging { // Create an instance of the class named by the given Java system property, or by // defaultClassName if the property is not set, and return it as a T def instantiateClass[T](propertyName: String, defaultClassName: String): T = { - val name = System.getProperty(propertyName, defaultClassName) + val name = conf.getOrElse(propertyName, defaultClassName) Class.forName(name, true, classLoader).newInstance().asInstanceOf[T] } val serializerManager = new SerializerManager val serializer = serializerManager.setDefault( - System.getProperty("spark.serializer", "org.apache.spark.serializer.JavaSerializer")) + conf.getOrElse("spark.serializer", "org.apache.spark.serializer.JavaSerializer")) val closureSerializer = serializerManager.get( - System.getProperty("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")) + conf.getOrElse("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")) def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = { if (isDriver) { logInfo("Registering " + name) Left(actorSystem.actorOf(Props(newActor), name = name)) } else { - val driverHost: String = System.getProperty("spark.driver.host", "localhost") - val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt + val driverHost: String = conf.getOrElse("spark.driver.host", "localhost") + val driverPort: Int = conf.getOrElse("spark.driver.port", "7077").toInt Utils.checkHost(driverHost, "Expected hostname") val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name) logInfo("Connecting to " + name + ": " + url) @@ -168,21 +171,21 @@ object SparkEnv extends Logging { val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", - new BlockManagerMasterActor(isLocal))) - val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer) + new BlockManagerMasterActor(isLocal, conf)), conf) + val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf) val connectionManager = blockManager.connectionManager - val broadcastManager = new BroadcastManager(isDriver) + val broadcastManager = new BroadcastManager(isDriver, conf) val cacheManager = new CacheManager(blockManager) // Have to assign trackerActor after initialization as MapOutputTrackerActor // requires the MapOutputTracker itself val mapOutputTracker = if (isDriver) { - new MapOutputTrackerMaster() + new MapOutputTrackerMaster(conf) } else { - new MapOutputTracker() + new MapOutputTracker(conf) } mapOutputTracker.trackerActor = registerOrLookup( "MapOutputTracker", @@ -193,12 +196,12 @@ object SparkEnv extends Logging { val httpFileServer = new HttpFileServer() httpFileServer.initialize() - System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) + conf.set("spark.fileserver.uri", httpFileServer.serverUri) val metricsSystem = if (isDriver) { - MetricsSystem.createMetricsSystem("driver") + MetricsSystem.createMetricsSystem("driver", conf) } else { - MetricsSystem.createMetricsSystem("executor") + MetricsSystem.createMetricsSystem("executor", conf) } metricsSystem.start() @@ -212,7 +215,7 @@ object SparkEnv extends Logging { } // Warn about deprecated spark.cache.class property - if (System.getProperty("spark.cache.class") != null) { + if (conf.getOrElse("spark.cache.class", null) != null) { logWarning("The spark.cache.class property is no longer being used! Specify storage " + "levels using the RDD.persist() method instead.") } @@ -231,6 +234,7 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, - metricsSystem) + metricsSystem, + conf) } } 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 ca42c76928..d6eacfe23e 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 @@ -41,7 +41,7 @@ private[spark] class PythonRDD[T: ClassTag]( accumulator: Accumulator[JList[Array[Byte]]]) extends RDD[Array[Byte]](parent) { - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt override def getPartitions = parent.partitions @@ -247,10 +247,10 @@ private class BytesToString extends org.apache.spark.api.java.function.Function[ */ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) extends AccumulatorParam[JList[Array[Byte]]] { - + import SparkContext.{globalConf => conf} Utils.checkHost(serverHost, "Expected hostname") - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index 43c18294c5..be99d229ef 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -32,7 +32,7 @@ abstract class Broadcast[T](private[spark] val id: Long) extends Serializable { } private[spark] -class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable { +class BroadcastManager(val _isDriver: Boolean, conf: SparkConf) extends Logging with Serializable { private var initialized = false private var broadcastFactory: BroadcastFactory = null @@ -43,14 +43,14 @@ class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable private def initialize() { synchronized { if (!initialized) { - val broadcastFactoryClass = System.getProperty( + val broadcastFactoryClass = conf.getOrElse( "spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") broadcastFactory = Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] // Initialize appropriate BroadcastFactory and BroadcastObject - broadcastFactory.initialize(isDriver) + broadcastFactory.initialize(isDriver, conf) initialized = true } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index 68bff75b90..fb161ce69d 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -17,6 +17,8 @@ package org.apache.spark.broadcast +import org.apache.spark.SparkConf + /** * An interface for all the broadcast implementations in Spark (to allow * multiple broadcast implementations). SparkContext uses a user-specified @@ -24,7 +26,7 @@ package org.apache.spark.broadcast * entire Spark job. */ private[spark] trait BroadcastFactory { - def initialize(isDriver: Boolean): Unit + def initialize(isDriver: Boolean, conf: SparkConf): Unit def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] def stop(): Unit } 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 47db720416..cecb8c228b 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit import it.unimi.dsi.fastutil.io.FastBufferedInputStream import it.unimi.dsi.fastutil.io.FastBufferedOutputStream -import org.apache.spark.{HttpServer, Logging, SparkEnv} +import org.apache.spark.{SparkConf, HttpServer, Logging, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashSet, Utils} @@ -64,7 +64,7 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea } private[spark] class HttpBroadcastFactory extends BroadcastFactory { - def initialize(isDriver: Boolean) { HttpBroadcast.initialize(isDriver) } + def initialize(isDriver: Boolean, conf: SparkConf) { HttpBroadcast.initialize(isDriver, conf) } def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = new HttpBroadcast[T](value_, isLocal, id) @@ -88,15 +88,16 @@ private object HttpBroadcast extends Logging { private lazy val compressionCodec = CompressionCodec.createCodec() - def initialize(isDriver: Boolean) { + def initialize(isDriver: Boolean, conf: SparkConf) { synchronized { if (!initialized) { - bufferSize = System.getProperty("spark.buffer.size", "65536").toInt - compress = System.getProperty("spark.broadcast.compress", "true").toBoolean + bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + compress = conf.getOrElse("spark.broadcast.compress", "true").toBoolean if (isDriver) { createServer() + conf.set("spark.httpBroadcast.uri", serverUri) } - serverUri = System.getProperty("spark.httpBroadcast.uri") + serverUri = conf.get("spark.httpBroadcast.uri") initialized = true } } @@ -118,7 +119,6 @@ private object HttpBroadcast extends Logging { server = new HttpServer(broadcastDir) server.start() serverUri = server.uri - System.setProperty("spark.httpBroadcast.uri", serverUri) logInfo("Broadcast server started at " + serverUri) } diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 073a0a5029..4a3801dc48 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -166,8 +166,9 @@ private object TorrentBroadcast extends Logging { private var initialized = false - - def initialize(_isDriver: Boolean) { + private var conf: SparkConf = null + def initialize(_isDriver: Boolean, conf: SparkConf) { + TorrentBroadcast.conf = conf //TODO: we might have to fix it in tests synchronized { if (!initialized) { initialized = true @@ -179,7 +180,7 @@ extends Logging { initialized = false } - val BLOCK_SIZE = System.getProperty("spark.broadcast.blockSize", "4096").toInt * 1024 + lazy val BLOCK_SIZE = conf.getOrElse("spark.broadcast.blockSize", "4096").toInt * 1024 def blockifyObject[T](obj: T): TorrentInfo = { val byteArray = Utils.serialize[T](obj) @@ -238,7 +239,7 @@ private[spark] case class TorrentInfo( private[spark] class TorrentBroadcastFactory extends BroadcastFactory { - def initialize(isDriver: Boolean) { TorrentBroadcast.initialize(isDriver) } + def initialize(isDriver: Boolean, conf: SparkConf) { TorrentBroadcast.initialize(isDriver, conf) } def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = new TorrentBroadcast[T](value_, isLocal, id) diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 19d393a0db..dda43dc018 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -26,7 +26,7 @@ private[spark] class ApplicationDescription( val appUiUrl: String) extends Serializable { - val user = System.getProperty("user.name", "") + val user = System.getProperty("user.name", "") override def toString: String = "ApplicationDescription(" + name + ")" } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index fc1537f796..1c979ac3e0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -67,8 +67,9 @@ class SparkHadoopUtil { } object SparkHadoopUtil { + import SparkContext.{globalConf => conf} private val hadoop = { - val yarnMode = java.lang.Boolean.valueOf(System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) + val yarnMode = java.lang.Boolean.valueOf(conf.getOrElse("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) if (yarnMode) { try { Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil").newInstance.asInstanceOf[SparkHadoopUtil] diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 953755e40d..9bbd635ab9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -19,20 +19,18 @@ package org.apache.spark.deploy.client import java.util.concurrent.TimeoutException -import scala.concurrent.duration._ import scala.concurrent.Await +import scala.concurrent.duration._ import akka.actor._ import akka.pattern.ask -import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} - -import org.apache.spark.{SparkException, Logging} +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master import org.apache.spark.util.AkkaUtils - /** * 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. @@ -43,7 +41,8 @@ private[spark] class Client( actorSystem: ActorSystem, masterUrls: Array[String], appDescription: ApplicationDescription, - listener: ClientListener) + listener: ClientListener, + conf: SparkConf) extends Logging { val REGISTRATION_TIMEOUT = 20.seconds @@ -178,7 +177,7 @@ private[spark] class Client( def stop() { if (actor != null) { try { - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(conf) val future = actor.ask(StopClient)(timeout) Await.result(future, timeout) } catch { 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 5b62d3ba6c..426cf524ae 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 @@ -18,7 +18,7 @@ package org.apache.spark.deploy.client import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.{Logging} +import org.apache.spark.{SparkContext, Logging} import org.apache.spark.deploy.{Command, ApplicationDescription} private[spark] object TestClient { @@ -45,11 +45,12 @@ private[spark] object TestClient { def main(args: Array[String]) { val url = args(0) - val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0) + val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, + conf = SparkContext.globalConf) val desc = new ApplicationDescription( "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored") val listener = new TestListener - val client = new Client(actorSystem, Array(url), desc, listener) + val client = new Client(actorSystem, Array(url), desc, listener, SparkContext.globalConf) client.start() actorSystem.awaitTermination() } 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 eebd0794b8..2c162c4fa2 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 @@ -29,7 +29,7 @@ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.{SparkContext, Logging, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.MasterMessages._ @@ -39,13 +39,13 @@ import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { import context.dispatcher - + val conf = SparkContext.globalConf val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 - val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retainedApplications", "200").toInt - val REAPER_ITERATIONS = System.getProperty("spark.dead.worker.persistence", "15").toInt - val RECOVERY_DIR = System.getProperty("spark.deploy.recoveryDirectory", "") - val RECOVERY_MODE = System.getProperty("spark.deploy.recoveryMode", "NONE") + val WORKER_TIMEOUT = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 + val RETAINED_APPLICATIONS = conf.getOrElse("spark.deploy.retainedApplications", "200").toInt + val REAPER_ITERATIONS = conf.getOrElse("spark.dead.worker.persistence", "15").toInt + val RECOVERY_DIR = conf.getOrElse("spark.deploy.recoveryDirectory", "") + val RECOVERY_MODE = conf.getOrElse("spark.deploy.recoveryMode", "NONE") var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] @@ -63,8 +63,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act Utils.checkHost(host, "Expected hostname") - val masterMetricsSystem = MetricsSystem.createMetricsSystem("master") - val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications") + val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf) + val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf) val masterSource = new MasterSource(this) val webUi = new MasterWebUI(this, webUiPort) @@ -86,7 +86,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "true").toBoolean + val spreadOutApps = conf.getOrElse("spark.deploy.spreadOut", "true").toBoolean override def preStart() { logInfo("Starting Spark master at " + masterUrl) @@ -103,7 +103,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act persistenceEngine = RECOVERY_MODE match { case "ZOOKEEPER" => logInfo("Persisting recovery state to ZooKeeper") - new ZooKeeperPersistenceEngine(SerializationExtension(context.system)) + new ZooKeeperPersistenceEngine(SerializationExtension(context.system), conf) case "FILESYSTEM" => logInfo("Persisting recovery state to directory: " + RECOVERY_DIR) new FileSystemPersistenceEngine(RECOVERY_DIR, SerializationExtension(context.system)) @@ -113,7 +113,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act leaderElectionAgent = RECOVERY_MODE match { case "ZOOKEEPER" => - context.actorOf(Props(classOf[ZooKeeperLeaderElectionAgent], self, masterUrl)) + context.actorOf(Props(classOf[ZooKeeperLeaderElectionAgent], self, masterUrl, conf)) case _ => context.actorOf(Props(classOf[MonarchyLeaderAgent], self)) } @@ -507,7 +507,7 @@ private[spark] object Master { val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r def main(argStrings: Array[String]) { - val args = new MasterArguments(argStrings) + val args = new MasterArguments(argStrings, SparkContext.globalConf) val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort) actorSystem.awaitTermination() } @@ -523,9 +523,10 @@ private[spark] object Master { } def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, + conf = SparkContext.globalConf) val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), actorName) - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(SparkContext.globalConf) val respFuture = actor.ask(RequestWebUIPort)(timeout) val resp = Await.result(respFuture, timeout).asInstanceOf[WebUIPortResponse] (actorSystem, boundPort, resp.webUIBoundPort) 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 9d89b455fb..7ce83f9c36 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 @@ -18,11 +18,12 @@ package org.apache.spark.deploy.master import org.apache.spark.util.{Utils, IntParam} +import org.apache.spark.SparkConf /** * Command-line parser for the master. */ -private[spark] class MasterArguments(args: Array[String]) { +private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 7077 var webUiPort = 8080 @@ -37,8 +38,8 @@ 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 + if (conf.get("master.ui.port") != null) { + webUiPort = conf.get("master.ui.port").toInt } parse(args.toList) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala index 6cc7fd2ff4..79d95b1a83 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala @@ -23,7 +23,7 @@ import org.apache.zookeeper._ import org.apache.zookeeper.Watcher.Event.KeeperState import org.apache.zookeeper.data.Stat -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} /** * Provides a Scala-side interface to the standard ZooKeeper client, with the addition of retry @@ -35,8 +35,9 @@ import org.apache.spark.Logging * Additionally, all commands sent to ZooKeeper will be retried until they either fail too many * times or a semantic exception is thrown (e.g., "node already exists"). */ -private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) extends Logging { - val ZK_URL = System.getProperty("spark.deploy.zookeeper.url", "") +private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher, + conf: SparkConf) extends Logging { + val ZK_URL = conf.getOrElse("spark.deploy.zookeeper.url", "") val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE val ZK_TIMEOUT_MILLIS = 30000 diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 7d535b08de..df5bb368a2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -21,16 +21,17 @@ import akka.actor.ActorRef import org.apache.zookeeper._ import org.apache.zookeeper.Watcher.Event.EventType -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.deploy.master.MasterMessages._ -private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String) +private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, + masterUrl: String, conf: SparkConf) extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging { - val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" + val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" private val watcher = new ZooKeeperWatcher() - private val zk = new SparkZooKeeperSession(this) + private val zk = new SparkZooKeeperSession(this, conf) private var status = LeadershipStatus.NOT_LEADER private var myLeaderFile: String = _ private var leaderUrl: String = _ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 825344b3bb..c55b720422 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -17,19 +17,19 @@ package org.apache.spark.deploy.master -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.zookeeper._ import akka.serialization.Serialization -class ZooKeeperPersistenceEngine(serialization: Serialization) +class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) extends PersistenceEngine with SparkZooKeeperWatcher with Logging { - val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/master_status" - val zk = new SparkZooKeeperSession(this) + val zk = new SparkZooKeeperSession(this, conf) zk.connect() 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 9ab594b682..ead35662fc 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 @@ -31,7 +31,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(master.conf) val host = Utils.localHostName() val port = requestedPort 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 87531b6719..75a6e75c78 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 @@ -25,23 +25,14 @@ import scala.collection.mutable.HashMap import scala.concurrent.duration._ import akka.actor._ -import akka.remote.{ DisassociatedEvent, RemotingLifecycleEvent} - -import org.apache.spark.{SparkException, Logging} +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ExecutorDescription, 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.{Utils, AkkaUtils} -import org.apache.spark.deploy.DeployMessages.WorkerStateResponse -import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed -import org.apache.spark.deploy.DeployMessages.KillExecutor -import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged -import org.apache.spark.deploy.DeployMessages.Heartbeat -import org.apache.spark.deploy.DeployMessages.RegisteredWorker -import org.apache.spark.deploy.DeployMessages.LaunchExecutor -import org.apache.spark.deploy.DeployMessages.RegisterWorker +import org.apache.spark.util.{AkkaUtils, Utils} /** * @param masterUrls Each url should look like spark://host:port. @@ -53,7 +44,8 @@ private[spark] class Worker( cores: Int, memory: Int, masterUrls: Array[String], - workDirPath: String = null) + workDirPath: String = null, + val conf: SparkConf) extends Actor with Logging { import context.dispatcher @@ -63,7 +55,7 @@ private[spark] class Worker( val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs // Send a heartbeat every (heartbeat timeout) / 4 milliseconds - val HEARTBEAT_MILLIS = System.getProperty("spark.worker.timeout", "60").toLong * 1000 / 4 + val HEARTBEAT_MILLIS = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 / 4 val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 @@ -92,7 +84,7 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 - val metricsSystem = MetricsSystem.createMetricsSystem("worker") + val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf) val workerSource = new WorkerSource(this) def coresFree: Int = cores - coresUsed @@ -275,6 +267,7 @@ private[spark] class Worker( } private[spark] object Worker { + import org.apache.spark.SparkContext.globalConf def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, @@ -287,9 +280,10 @@ private[spark] object Worker { : (ActorSystem, Int) = { // The LocalSparkCluster runs multiple local sparkWorkerX actor systems val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("") - val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, + conf = globalConf) actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory, - masterUrls, workDir), name = "Worker") + masterUrls, workDir, globalConf), name = "Worker") (actorSystem, boundPort) } 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 40d6bdb3fd..ec47ba1b56 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 @@ -22,7 +22,7 @@ import java.io.File import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.worker.Worker import org.apache.spark.ui.{JettyUtils, UIUtils} import org.apache.spark.ui.JettyUtils._ @@ -34,10 +34,10 @@ import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) extends Logging { - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(worker.conf) val host = Utils.localHostName() val port = requestedPort.getOrElse( - System.getProperty("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) + worker.conf.getOrElse("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) var server: Option[Server] = None var boundPort: Option[Int] = None diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index debbdd4c44..c8319f6f6e 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import akka.actor._ import akka.remote._ -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, Logging} import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{Utils, AkkaUtils} @@ -98,10 +98,10 @@ private[spark] object CoarseGrainedExecutorBackend { // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor // before getting started with all our system properties, etc val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, - indestructible = true) + indestructible = true, conf = SparkContext.globalConf) // set it val sparkHostPort = hostname + ":" + boundPort - System.setProperty("spark.hostPort", sparkHostPort) +// conf.set("spark.hostPort", sparkHostPort) actorSystem.actorOf( Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores), name = "Executor") 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 0f19d7a96b..70fc30e993 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -57,17 +57,17 @@ private[spark] class Executor( // Make sure the local hostname we report matches the cluster scheduler's name for this host Utils.setCustomHostname(slaveHostname) - + val conf = new SparkConf(false) // Set spark.* system properties from executor arg for ((key, value) <- properties) { - System.setProperty(key, value) + conf.set(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.getenv("SPARK_YARN_MODE"))) { - System.setProperty("spark.local.dir", getYarnLocalDirs()) + conf.set("spark.local.dir", getYarnLocalDirs()) } // Create our ClassLoader and set it on this thread @@ -108,7 +108,7 @@ private[spark] class Executor( // Initialize Spark environment (using system properties read above) private val env = { if (!isLocal) { - val _env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, + val _env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, conf, isDriver = false, isLocal = false) SparkEnv.set(_env) _env.metricsSystem.registerSource(executorSource) @@ -303,7 +303,7 @@ private[spark] class Executor( * 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") + val classUri = conf.getOrElse("spark.repl.class.uri", null) if (classUri != null) { logInfo("Using REPL class URI: " + classUri) try { 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 570a979b56..8ef5019b6c 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -22,6 +22,7 @@ import java.io.{InputStream, OutputStream} import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} +import org.apache.spark.SparkConf /** @@ -37,15 +38,16 @@ trait CompressionCodec { private[spark] object CompressionCodec { - + import org.apache.spark.SparkContext.globalConf def createCodec(): CompressionCodec = { createCodec(System.getProperty( "spark.io.compression.codec", classOf[LZFCompressionCodec].getName)) } def createCodec(codecName: String): CompressionCodec = { - Class.forName(codecName, true, Thread.currentThread.getContextClassLoader) - .newInstance().asInstanceOf[CompressionCodec] + val ctor = Class.forName(codecName, true, Thread.currentThread.getContextClassLoader) + .getConstructor(classOf[SparkConf]) + ctor.newInstance(globalConf).asInstanceOf[CompressionCodec] } } @@ -53,7 +55,7 @@ private[spark] object CompressionCodec { /** * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. */ -class LZFCompressionCodec extends CompressionCodec { +class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { new LZFOutputStream(s).setFinishBlockOnFlush(true) @@ -67,10 +69,10 @@ class LZFCompressionCodec extends 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 { +class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = System.getProperty("spark.io.compression.snappy.block.size", "32768").toInt + val blockSize = conf.getOrElse("spark.io.compression.snappy.block.size", "32768").toInt new SnappyOutputStream(s, blockSize) } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index bec0c83be8..ac29816f19 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.metrics.sink.{MetricsServlet, Sink} import org.apache.spark.metrics.source.Source @@ -62,10 +62,11 @@ import org.apache.spark.metrics.source.Source * * [options] is the specific property of this source or sink. */ -private[spark] class MetricsSystem private (val instance: String) extends Logging { +private[spark] class MetricsSystem private (val instance: String, + conf: SparkConf) extends Logging { initLogging() - val confFile = System.getProperty("spark.metrics.conf") + val confFile = conf.getOrElse("spark.metrics.conf", null) val metricsConfig = new MetricsConfig(Option(confFile)) val sinks = new mutable.ArrayBuffer[Sink] @@ -159,5 +160,6 @@ private[spark] object MetricsSystem { } } - def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) + def createMetricsSystem(instance: String, conf: SparkConf): MetricsSystem = + new MetricsSystem(instance, conf) } 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 703bc6a9ca..3e902f8ac5 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -37,7 +37,7 @@ import scala.concurrent.duration._ import org.apache.spark.util.Utils -private[spark] class ConnectionManager(port: Int) extends Logging { +private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Logging { class MessageStatus( val message: Message, @@ -54,22 +54,22 @@ private[spark] class ConnectionManager(port: Int) extends Logging { private val selector = SelectorProvider.provider.openSelector() private val handleMessageExecutor = new ThreadPoolExecutor( - System.getProperty("spark.core.connection.handler.threads.min","20").toInt, - System.getProperty("spark.core.connection.handler.threads.max","60").toInt, - System.getProperty("spark.core.connection.handler.threads.keepalive","60").toInt, TimeUnit.SECONDS, + conf.getOrElse("spark.core.connection.handler.threads.min", "20").toInt, + conf.getOrElse("spark.core.connection.handler.threads.max", "60").toInt, + conf.getOrElse("spark.core.connection.handler.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) private val handleReadWriteExecutor = new ThreadPoolExecutor( - System.getProperty("spark.core.connection.io.threads.min","4").toInt, - System.getProperty("spark.core.connection.io.threads.max","32").toInt, - System.getProperty("spark.core.connection.io.threads.keepalive","60").toInt, TimeUnit.SECONDS, + conf.getOrElse("spark.core.connection.io.threads.min", "4").toInt, + conf.getOrElse("spark.core.connection.io.threads.max", "32").toInt, + conf.getOrElse("spark.core.connection.io.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : which should be executed asap private val handleConnectExecutor = new ThreadPoolExecutor( - System.getProperty("spark.core.connection.connect.threads.min","1").toInt, - System.getProperty("spark.core.connection.connect.threads.max","8").toInt, - System.getProperty("spark.core.connection.connect.threads.keepalive","60").toInt, TimeUnit.SECONDS, + conf.getOrElse("spark.core.connection.connect.threads.min", "1").toInt, + conf.getOrElse("spark.core.connection.connect.threads.max", "8").toInt, + conf.getOrElse("spark.core.connection.connect.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) private val serverChannel = ServerSocketChannel.open() @@ -593,8 +593,10 @@ private[spark] class ConnectionManager(port: Int) extends Logging { private[spark] object ConnectionManager { + import SparkContext.globalConf + def main(args: Array[String]) { - val manager = new ConnectionManager(9999) + val manager = new ConnectionManager(9999, globalConf) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { println("Received [" + msg + "] from [" + id + "]") None diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 781715108b..4ca3cd390b 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -21,9 +21,9 @@ import java.nio.ByteBuffer import java.net.InetAddress private[spark] object ReceiverTest { - + import org.apache.spark.SparkContext.globalConf def main(args: Array[String]) { - val manager = new ConnectionManager(9999) + val manager = new ConnectionManager(9999, globalConf) println("Started connection manager with id = " + manager.id) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index 777574980f..11c21fc1d5 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import java.net.InetAddress private[spark] object SenderTest { - + import org.apache.spark.SparkContext.globalConf def main(args: Array[String]) { if (args.length < 2) { @@ -33,7 +33,7 @@ private[spark] object SenderTest { val targetPort = args(1).toInt val targetConnectionManagerId = new ConnectionManagerId(targetHost, targetPort) - val manager = new ConnectionManager(0) + val manager = new ConnectionManager(0, globalConf) println("Started connection manager with id = " + manager.id) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala index b1e1576dad..81b3104afd 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala @@ -23,20 +23,20 @@ import io.netty.buffer.ByteBuf import io.netty.channel.ChannelHandlerContext import io.netty.util.CharsetUtil -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, SparkConf, Logging} import org.apache.spark.network.ConnectionManagerId import scala.collection.JavaConverters._ import org.apache.spark.storage.BlockId -private[spark] class ShuffleCopier extends Logging { +private[spark] class ShuffleCopier(conf: SparkConf) extends Logging { def getBlock(host: String, port: Int, blockId: BlockId, resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val connectTimeout = System.getProperty("spark.shuffle.netty.connect.timeout", "60000").toInt + val connectTimeout = conf.getOrElse("spark.shuffle.netty.connect.timeout", "60000").toInt val fc = new FileClient(handler, connectTimeout) try { @@ -107,7 +107,7 @@ private[spark] object ShuffleCopier extends Logging { val tasks = (for (i <- Range(0, threads)) yield { Executors.callable(new Runnable() { def run() { - val copier = new ShuffleCopier() + val copier = new ShuffleCopier(SparkContext.globalConf) copier.getBlock(host, port, blockId, echoResultCollectCallBack) } }) diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index a712ef1c27..9fbe002748 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -75,6 +75,8 @@ class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String) private[spark] object CheckpointRDD extends Logging { + import SparkContext.{globalConf => conf} + def splitIdToFile(splitId: Int): String = { "part-%05d".format(splitId) } @@ -92,7 +94,7 @@ private[spark] object CheckpointRDD extends Logging { throw new IOException("Checkpoint failed: temporary path " + tempOutputPath + " already exists") } - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt val fileOutputStream = if (blockSize < 0) { fs.create(tempOutputPath, false, bufferSize) @@ -122,7 +124,7 @@ private[spark] object CheckpointRDD extends Logging { def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = { val env = SparkEnv.get val fs = path.getFileSystem(SparkHadoopUtil.get.newConfiguration()) - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt val fileInputStream = fs.open(path, bufferSize) val serializer = env.serializer.newInstance() val deserializeStream = serializer.deserializeStream(fileInputStream) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index ea45566ad1..f8b1a6932e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -81,6 +81,7 @@ abstract class RDD[T: ClassTag]( def this(@transient oneParent: RDD[_]) = this(oneParent.context , List(new OneToOneDependency(oneParent))) + private[spark] def conf = sc.conf // ======================================================================= // Methods that should be implemented by subclasses of RDD // ======================================================================= 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 60927831a1..3f55cd5642 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -41,7 +41,7 @@ import org.apache.spark.storage.StorageLevel class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { - def this() = this(System.getProperty("user.name", ""), + def this() = this(System.getProperty("user.name", ""), String.valueOf(System.currentTimeMillis())) private val logDir = diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index 356fe56bf3..9002d33cda 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler import java.io.{FileInputStream, InputStream} import java.util.{NoSuchElementException, Properties} -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import scala.xml.XML @@ -49,10 +49,10 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) } } -private[spark] class FairSchedulableBuilder(val rootPool: Pool) +private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) extends SchedulableBuilder with Logging { - val schedulerAllocFile = Option(System.getProperty("spark.scheduler.allocation.file")) + val schedulerAllocFile = Option(conf.get("spark.scheduler.allocation.file")) val DEFAULT_SCHEDULER_FILE = "fairscheduler.xml" val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.pool" val DEFAULT_POOL_NAME = "default" diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala index 66ab8ea4cd..7e231ec44c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala @@ -49,11 +49,12 @@ private[spark] class ClusterScheduler(val sc: SparkContext) extends TaskScheduler with Logging { + val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong + val SPECULATION_INTERVAL = conf.getOrElse("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 + val STARVATION_TIMEOUT = conf.getOrElse("spark.starvation.timeout", "15000").toLong // ClusterTaskSetManagers are not thread safe, so any access to one should be synchronized // on this class. @@ -90,7 +91,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var rootPool: Pool = null // default scheduler is FIFO val schedulingMode: SchedulingMode = SchedulingMode.withName( - System.getProperty("spark.scheduler.mode", "FIFO")) + conf.getOrElse("spark.scheduler.mode", "FIFO")) // This is a var so that we can reset it for testing purposes. private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) @@ -108,7 +109,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) case SchedulingMode.FIFO => new FIFOSchedulableBuilder(rootPool) case SchedulingMode.FAIR => - new FairSchedulableBuilder(rootPool) + new FairSchedulableBuilder(rootPool, conf) } } schedulableBuilder.buildPools() @@ -119,7 +120,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) override def start() { backend.start() - if (System.getProperty("spark.speculation", "false").toBoolean) { + if (conf.getOrElse("spark.speculation", "false").toBoolean) { logInfo("Starting speculative execution thread") import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, 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 bf494aa64d..398b0cefbf 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 @@ -50,15 +50,16 @@ private[spark] class ClusterTaskSetManager( extends TaskSetManager with Logging { + val conf = sched.sc.conf // CPUs to request per task - val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toInt + val CPUS_PER_TASK = conf.getOrElse("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 + val MAX_TASK_FAILURES = conf.getOrElse("spark.task.maxFailures", "4").toInt // Quantile of tasks at which to start speculation - val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble - val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble + val SPECULATION_QUANTILE = conf.getOrElse("spark.speculation.quantile", "0.75").toDouble + val SPECULATION_MULTIPLIER = conf.getOrElse("spark.speculation.multiplier", "1.5").toDouble // Serializer for closures and tasks. val env = SparkEnv.get @@ -117,7 +118,7 @@ private[spark] class ClusterTaskSetManager( // How frequently to reprint duplicate exceptions in full, in milliseconds val EXCEPTION_PRINT_INTERVAL = - System.getProperty("spark.logging.exceptionPrintInterval", "10000").toLong + conf.getOrElse("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 @@ -677,14 +678,14 @@ private[spark] class ClusterTaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = System.getProperty("spark.locality.wait", "3000") + val defaultWait = conf.getOrElse("spark.locality.wait", "3000") level match { case TaskLocality.PROCESS_LOCAL => - System.getProperty("spark.locality.wait.process", defaultWait).toLong + conf.getOrElse("spark.locality.wait.process", defaultWait).toLong case TaskLocality.NODE_LOCAL => - System.getProperty("spark.locality.wait.node", defaultWait).toLong + conf.getOrElse("spark.locality.wait.node", defaultWait).toLong case TaskLocality.RACK_LOCAL => - System.getProperty("spark.locality.wait.rack", defaultWait).toLong + conf.getOrElse("spark.locality.wait.rack", defaultWait).toLong case TaskLocality.ANY => 0L } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 7e22c843bf..40555903ac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ +import scala.util.Try import akka.actor._ import akka.pattern.ask @@ -46,8 +47,8 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) - - private val timeout = AkkaUtils.askTimeout + val conf = scheduler.sc.conf + private val timeout = AkkaUtils.askTimeout(conf) class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { private val executorActor = new HashMap[String, ActorRef] @@ -61,7 +62,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) // Periodically revive offers to allow delay scheduling to work - val reviveInterval = System.getProperty("spark.scheduler.revive.interval", "1000").toLong + val reviveInterval = conf.getOrElse("spark.scheduler.revive.interval", "1000").toLong import context.dispatcher context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } @@ -162,7 +163,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac override def start() { val properties = new ArrayBuffer[(String, String)] - val iterator = System.getProperties.entrySet.iterator + val iterator = scheduler.sc.conf.getAllConfiguration while (iterator.hasNext) { val entry = iterator.next val (key, value) = (entry.getKey.toString, entry.getValue.toString) @@ -170,6 +171,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac properties += ((key, value)) } } + //TODO (prashant) send conf instead of properties driverActor = actorSystem.actorOf( Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) } @@ -208,7 +210,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac driverActor ! KillTask(taskId, executorId) } - override def defaultParallelism() = Option(System.getProperty("spark.default.parallelism")) + override def defaultParallelism() = Try(conf.get("spark.default.parallelism")).toOption .map(_.toInt).getOrElse(math.max(totalCoreCount.get(), 2)) // Called by subclasses when notified of a lost worker diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index e8fecec4a6..d01329b2b3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -31,13 +31,13 @@ private[spark] class SimrSchedulerBackend( val tmpPath = new Path(driverFilePath + "_tmp") val filePath = new Path(driverFilePath) - val maxCores = System.getProperty("spark.simr.executor.cores", "1").toInt + val maxCores = conf.getOrElse("spark.simr.executor.cores", "1").toInt override def start() { super.start() val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), + sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val conf = new Configuration() 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 7127a72d6d..d6b8ac2d57 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 @@ -36,14 +36,14 @@ private[spark] class SparkDeploySchedulerBackend( var stopping = false var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ - val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt + val maxCores = conf.getOrElse("spark.cores.max", Int.MaxValue.toString).toInt override def start() { super.start() // The endpoint for executors to talk to us val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), + conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") val command = Command( @@ -52,7 +52,7 @@ private[spark] class SparkDeploySchedulerBackend( val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, "http://" + sc.ui.appUIAddress) - client = new Client(sc.env.actorSystem, masters, appDesc, this) + client = new Client(sc.env.actorSystem, masters, appDesc, this, conf) client.start() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala index e68c527713..ff6cc37f1d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala @@ -31,7 +31,8 @@ import org.apache.spark.util.Utils */ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler) extends Logging { - private val THREADS = System.getProperty("spark.resultGetter.threads", "4").toInt + + private val THREADS = sparkEnv.conf.getOrElse("spark.resultGetter.threads", "4").toInt private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( THREADS, "Result resolver thread") diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 84fe3094cc..2a3b0e15f7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -61,7 +61,7 @@ private[spark] class CoarseMesosSchedulerBackend( var driver: SchedulerDriver = null // Maximum number of cores to acquire (TODO: we'll need more flexible controls here) - val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt + val maxCores = conf.getOrElse("spark.cores.max", Int.MaxValue.toString).toInt // Cores we have acquired with each Mesos task ID val coresByTaskId = new HashMap[Int, Int] @@ -76,7 +76,7 @@ private[spark] class CoarseMesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) - val extraCoresPerSlave = System.getProperty("spark.mesos.extra.cores", "0").toInt + val extraCoresPerSlave = conf.getOrElse("spark.mesos.extra.cores", "0").toInt var nextMesosTaskId = 0 @@ -121,10 +121,10 @@ private[spark] class CoarseMesosSchedulerBackend( val command = CommandInfo.newBuilder() .setEnvironment(environment) val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), - System.getProperty("spark.driver.port"), + conf.get("spark.driver.host"), + conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val uri = System.getProperty("spark.executor.uri") + val uri = conf.get("spark.executor.uri") if (uri == null) { val runScript = new File(sparkHome, "spark-class").getCanonicalPath command.setValue( diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 50cbc2ca92..9bb92b4f01 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -101,7 +101,7 @@ private[spark] class MesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val uri = System.getProperty("spark.executor.uri") + val uri = sc.conf.get("spark.executor.uri") if (uri == null) { command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath) } else { @@ -341,5 +341,5 @@ private[spark] class MesosSchedulerBackend( } // TODO: query Mesos for number of cores - override def defaultParallelism() = System.getProperty("spark.default.parallelism", "8").toInt + override def defaultParallelism() = sc.conf.getOrElse("spark.default.parallelism", "8").toInt } 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 01e95162c0..6069c1db3a 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 @@ -80,6 +80,7 @@ private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val with Logging { val env = SparkEnv.get + val conf = env.conf val attemptId = new AtomicInteger var dagScheduler: DAGScheduler = null @@ -91,7 +92,7 @@ private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null val schedulingMode: SchedulingMode = SchedulingMode.withName( - System.getProperty("spark.scheduler.mode", "FIFO")) + conf.getOrElse("spark.scheduler.mode", "FIFO")) val activeTaskSets = new HashMap[String, LocalTaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] @@ -106,7 +107,7 @@ private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val case SchedulingMode.FIFO => new FIFOSchedulableBuilder(rootPool) case SchedulingMode.FAIR => - new FairSchedulableBuilder(rootPool) + new FairSchedulableBuilder(rootPool, conf) } } schedulableBuilder.buildPools() diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index e748c2275d..17cec81038 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -25,18 +25,20 @@ import com.esotericsoftware.kryo.{KryoException, Kryo} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.twitter.chill.{EmptyScalaKryoInstantiator, AllScalaRegistrar} -import org.apache.spark.{SerializableWritable, Logging} +import org.apache.spark.{SparkContext, SparkConf, SerializableWritable, Logging} import org.apache.spark.broadcast.HttpBroadcast import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ +import scala.util.Try /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. */ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging { + private val conf = SparkContext.globalConf private val bufferSize = { - System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 + conf.getOrElse("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 } def newKryoOutput() = new KryoOutput(bufferSize) @@ -48,7 +50,7 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. // Do this before we invoke the user registrator so the user registrator can override this. - kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean) + kryo.setReferences(conf.getOrElse("spark.kryo.referenceTracking", "true").toBoolean) for (cls <- KryoSerializer.toRegister) kryo.register(cls) @@ -58,7 +60,7 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging // Allow the user to register their own classes by setting spark.kryo.registrator try { - Option(System.getProperty("spark.kryo.registrator")).foreach { regCls => + Try(conf.get("spark.kryo.registrator")).toOption.foreach { regCls => logDebug("Running user registrator: " + regCls) val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] reg.registerClasses(kryo) 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 e51c5b30a3..ee2ae471a9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -312,7 +312,7 @@ object BlockFetcherIterator { logDebug("Sending request for %d blocks (%s) from %s".format( req.blocks.size, Utils.bytesToString(req.size), req.address.host)) val cmId = new ConnectionManagerId(req.address.host, req.address.nettyPort) - val cpier = new ShuffleCopier + val cpier = new ShuffleCopier(blockManager.conf) cpier.getBlocks(cmId, req.blocks, putResult) logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) } @@ -327,7 +327,7 @@ object BlockFetcherIterator { fetchRequestsSync.put(request) } - copiers = startCopiers(System.getProperty("spark.shuffle.copier.threads", "6").toInt) + copiers = startCopiers(conf.getOrElse("spark.shuffle.copier.threads", "6").toInt) logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) 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 19a025a329..ffd166e93a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -30,7 +30,7 @@ import scala.concurrent.duration._ import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} -import org.apache.spark.{Logging, SparkEnv, SparkException} +import org.apache.spark.{SparkConf, Logging, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer @@ -43,12 +43,13 @@ private[spark] class BlockManager( actorSystem: ActorSystem, val master: BlockManagerMaster, val defaultSerializer: Serializer, - maxMemory: Long) + maxMemory: Long, + val conf: SparkConf) extends Logging { val shuffleBlockManager = new ShuffleBlockManager(this) val diskBlockManager = new DiskBlockManager(shuffleBlockManager, - System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir"))) + conf.getOrElse("spark.local.dir", System.getProperty("java.io.tmpdir"))) private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] @@ -57,12 +58,12 @@ private[spark] class BlockManager( // If we use Netty for shuffle, start a new Netty-based shuffle sender service. private val nettyPort: Int = { - val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean - val nettyPortConfig = System.getProperty("spark.shuffle.sender.port", "0").toInt + val useNetty = conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean + val nettyPortConfig = conf.getOrElse("spark.shuffle.sender.port", "0").toInt if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 } - val connectionManager = new ConnectionManager(0) + val connectionManager = new ConnectionManager(0, conf) implicit val futureExecContext = connectionManager.futureExecContext val blockManagerId = BlockManagerId( @@ -71,14 +72,14 @@ private[spark] class BlockManager( // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory // for receiving shuffle outputs) val maxBytesInFlight = - System.getProperty("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 + conf.getOrElse("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 // Whether to compress broadcast variables that are stored - val compressBroadcast = System.getProperty("spark.broadcast.compress", "true").toBoolean + val compressBroadcast = conf.getOrElse("spark.broadcast.compress", "true").toBoolean // Whether to compress shuffle output that are stored - val compressShuffle = System.getProperty("spark.shuffle.compress", "true").toBoolean + val compressShuffle = conf.getOrElse("spark.shuffle.compress", "true").toBoolean // Whether to compress RDD partitions that are stored serialized - val compressRdds = System.getProperty("spark.rdd.compress", "false").toBoolean + val compressRdds = conf.getOrElse("spark.rdd.compress", "false").toBoolean val heartBeatFrequency = BlockManager.getHeartBeatFrequencyFromSystemProperties @@ -115,8 +116,8 @@ private[spark] class BlockManager( * Construct a BlockManager with a memory limit set based on system properties. */ def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer) = { - this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties) + serializer: Serializer, conf: SparkConf) = { + this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties, conf) } /** @@ -439,7 +440,7 @@ private[spark] class BlockManager( : BlockFetcherIterator = { val iter = - if (System.getProperty("spark.shuffle.use.netty", "false").toBoolean) { + if (conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean) { new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) } else { new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer) @@ -465,7 +466,8 @@ private[spark] class BlockManager( def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int) : BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) - new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream) + val syncWrites = conf.getOrElse("spark.shuffle.sync", "false").toBoolean + new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites) } /** @@ -856,19 +858,19 @@ private[spark] class BlockManager( private[spark] object BlockManager extends Logging { - + import org.apache.spark.SparkContext.{globalConf => conf} val ID_GENERATOR = new IdGenerator def getMaxMemoryFromSystemProperties: Long = { - val memoryFraction = System.getProperty("spark.storage.memoryFraction", "0.66").toDouble + val memoryFraction = conf.getOrElse("spark.storage.memoryFraction", "0.66").toDouble (Runtime.getRuntime.maxMemory * memoryFraction).toLong } def getHeartBeatFrequencyFromSystemProperties: Long = - System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 + conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 def getDisableHeartBeatsForTesting: Boolean = - System.getProperty("spark.test.disableBlockManagerHeartBeat", "false").toBoolean + conf.getOrElse("spark.test.disableBlockManagerHeartBeat", "false").toBoolean /** * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index e1d68ef592..fde7d63a68 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -23,19 +23,20 @@ import scala.concurrent.ExecutionContext.Implicits.global import akka.actor._ import akka.pattern.ask -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.{SparkConf, Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.AkkaUtils private[spark] -class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection]) extends Logging { +class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection], + conf: SparkConf) extends Logging { - val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt - val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt + val AKKA_RETRY_ATTEMPTS: Int = conf.getOrElse("spark.akka.num.retries", "3").toInt + val AKKA_RETRY_INTERVAL_MS: Int = conf.getOrElse("spark.akka.retry.wait", "3000").toInt val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" - val timeout = AkkaUtils.askTimeout + val timeout = AkkaUtils.askTimeout(conf) /** Remove a dead executor from the driver actor. This is only called on the driver side. */ def removeExecutor(execId: String) { 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 21022e1cfb..05502e4451 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -27,7 +27,7 @@ import scala.concurrent.duration._ import akka.actor.{Actor, ActorRef, Cancellable} import akka.pattern.ask -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.{SparkConf, Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -36,7 +36,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} * all slaves' block managers. */ private[spark] -class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { +class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Actor with Logging { // Mapping from block manager id to the block manager's information. private val blockManagerInfo = @@ -48,14 +48,14 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { // Mapping from block id to the set of block managers that have the block. private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]] - private val akkaTimeout = AkkaUtils.askTimeout + private val akkaTimeout = AkkaUtils.askTimeout(conf) initLogging() - val slaveTimeout = System.getProperty("spark.storage.blockManagerSlaveTimeoutMs", + val slaveTimeout = conf.getOrElse("spark.storage.blockManagerSlaveTimeoutMs", "" + (BlockManager.getHeartBeatFrequencyFromSystemProperties * 3)).toLong - val checkTimeoutInterval = System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", + val checkTimeoutInterval = conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong var timeoutCheckingTask: Cancellable = null diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index b4451fc7b8..61e63c60d5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -74,7 +74,8 @@ class DiskBlockObjectWriter( file: File, serializer: Serializer, bufferSize: Int, - compressStream: OutputStream => OutputStream) + compressStream: OutputStream => OutputStream, + syncWrites: Boolean) extends BlockObjectWriter(blockId) with Logging { @@ -97,8 +98,6 @@ class DiskBlockObjectWriter( override def flush() = out.flush() } - private val syncWrites = System.getProperty("spark.shuffle.sync", "false").toBoolean - /** The file channel, used for repositioning / truncating the file. */ private var channel: FileChannel = null private var bs: OutputStream = null diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index fcd2e97982..8f528babd4 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -38,7 +38,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD extends PathResolver with Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 - private val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt + private val subDirsPerLocalDir = shuffleManager.conf.getOrElse("spark.diskStore.subDirectories", "64").toInt // Create one local directory for each path mentioned in spark.local.dir; then, inside this // directory, create multiple subdirectories that we will hash files into, in order to avoid diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index e828e1d1c5..850d3178dd 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -27,6 +27,8 @@ import org.apache.spark.serializer.Serializer import org.apache.spark.util.{MetadataCleanerType, MetadataCleaner, TimeStampedHashMap} import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} import org.apache.spark.storage.ShuffleBlockManager.ShuffleFileGroup +import scala.util.Try +import org.apache.spark.SparkConf /** A group of writers for a ShuffleMapTask, one writer per reducer. */ private[spark] trait ShuffleWriterGroup { @@ -59,12 +61,13 @@ private[spark] trait ShuffleWriterGroup { */ private[spark] class ShuffleBlockManager(blockManager: BlockManager) { + def conf = blockManager.conf // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. // TODO: Remove this once the shuffle file consolidation feature is stable. val consolidateShuffleFiles = - System.getProperty("spark.shuffle.consolidateFiles", "false").toBoolean + conf.getOrElse("spark.shuffle.consolidateFiles", "false").toBoolean - private val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 + private val bufferSize = conf.getOrElse("spark.shuffle.file.buffer.kb", "100").toInt * 1024 /** * Contains all the state related to a particular shuffle. This includes a pool of unused 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 a8db37ded1..b3b3893393 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -22,6 +22,7 @@ import akka.actor._ import java.util.concurrent.ArrayBlockingQueue import util.Random import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.SparkContext /** * This class tests the BlockManager and MemoryStore for thread safety and @@ -91,11 +92,12 @@ private[spark] object ThreadingTest { def main(args: Array[String]) { System.setProperty("spark.kryoserializer.buffer.mb", "1") val actorSystem = ActorSystem("test") + val conf = SparkContext.globalConf val serializer = new KryoSerializer val blockManagerMaster = new BlockManagerMaster( - Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))) + Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf) val blockManager = new BlockManager( - "", actorSystem, blockManagerMaster, serializer, 1024 * 1024) + "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.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 f1d86c0221..0ce8d9c8c4 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) - val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt + val port = sc.conf.getOrElse("spark.ui.port", SparkUI.DEFAULT_PORT).toInt var boundPort: Option[Int] = None var server: Option[Server] = None diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index fcd1b518d0..14751e8e8e 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -30,6 +30,8 @@ import org.apache.spark.scheduler.SchedulingMode * Usage: ./run spark.ui.UIWorkloadGenerator [master] */ private[spark] object UIWorkloadGenerator { + + import SparkContext.{globalConf => conf} val NUM_PARTITIONS = 100 val INTER_JOB_WAIT_MS = 5000 @@ -43,7 +45,7 @@ private[spark] object UIWorkloadGenerator { val appName = "Spark UI Tester" if (schedulingMode == SchedulingMode.FAIR) { - System.setProperty("spark.scheduler.mode", "FAIR") + conf.set("spark.scheduler.mode", "FAIR") } val sc = new SparkContext(master, appName) diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index c5bf2acc9e..b637d37517 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -63,7 +63,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) { UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties, fixedWidth = true) val classPathEntries = classPathProperty._2 - .split(System.getProperty("path.separator", ":")) + .split(sc.conf.getOrElse("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")} 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 6b854740d6..f01a1380b9 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 @@ -33,7 +33,7 @@ import org.apache.spark.scheduler._ */ 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 + val RETAINED_STAGES = sc.conf.getOrElse("spark.ui.retained_stages", "1000").toInt val DEFAULT_POOL_NAME = "default" val stageIdToPool = new HashMap[Int, String]() diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 1c8b51b8bc..76febd5702 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -21,6 +21,7 @@ import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} import com.typesafe.config.ConfigFactory +import org.apache.spark.SparkConf /** * Various utility classes for working with Akka. @@ -37,22 +38,22 @@ private[spark] object AkkaUtils { * If indestructible is set to true, the Actor System will continue running in the event * of a fatal exception. This is used by [[org.apache.spark.executor.Executor]]. */ - def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false) - : (ActorSystem, Int) = { + def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false, + conf: SparkConf): (ActorSystem, Int) = { - val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt - val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt + val akkaThreads = conf.getOrElse("spark.akka.threads", "4").toInt + val akkaBatchSize = conf.getOrElse("spark.akka.batchSize", "15").toInt - val akkaTimeout = System.getProperty("spark.akka.timeout", "100").toInt + val akkaTimeout = conf.getOrElse("spark.akka.timeout", "100").toInt - val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt + val akkaFrameSize = conf.getOrElse("spark.akka.frameSize", "10").toInt val lifecycleEvents = - if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" + if (conf.getOrElse("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "600").toInt + val akkaHeartBeatPauses = conf.getOrElse("spark.akka.heartbeat.pauses", "600").toInt val akkaFailureDetector = - System.getProperty("spark.akka.failure-detector.threshold", "300.0").toDouble - val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "1000").toInt + conf.getOrElse("spark.akka.failure-detector.threshold", "300.0").toDouble + val akkaHeartBeatInterval = conf.getOrElse("spark.akka.heartbeat.interval", "1000").toInt val akkaConf = ConfigFactory.parseString( s""" @@ -87,7 +88,7 @@ private[spark] object AkkaUtils { } /** Returns the default Spark timeout to use for Akka ask operations. */ - def askTimeout: FiniteDuration = { - Duration.create(System.getProperty("spark.akka.askTimeout", "30").toLong, "seconds") + def askTimeout(conf: SparkConf): FiniteDuration = { + Duration.create(conf.getOrElse("spark.akka.askTimeout", "30").toLong, "seconds") } } diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 7b41ef89f1..bf71d17a21 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -18,7 +18,7 @@ package org.apache.spark.util import java.util.{TimerTask, Timer} -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, Logging} /** @@ -66,21 +66,21 @@ object MetadataCleanerType extends Enumeration { } object MetadataCleaner { - + private val conf = SparkContext.globalConf // using only sys props for now : so that workers can also get to it while preserving earlier behavior. - def getDelaySeconds = System.getProperty("spark.cleaner.ttl", "-1").toInt + def getDelaySeconds = conf.getOrElse("spark.cleaner.ttl", "3500").toInt //TODO: this is to fix tests for time being def getDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType): Int = { - System.getProperty(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds.toString).toInt + conf.getOrElse(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds.toString).toInt } def setDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType, delay: Int) { - System.setProperty(MetadataCleanerType.systemProperty(cleanerType), delay.toString) + conf.set(MetadataCleanerType.systemProperty(cleanerType), delay.toString) } def setDelaySeconds(delay: Int, resetAll: Boolean = true) { // override for all ? - System.setProperty("spark.cleaner.ttl", delay.toString) + conf.set("spark.cleaner.ttl", delay.toString) if (resetAll) { for (cleanerType <- MetadataCleanerType.values) { System.clearProperty(MetadataCleanerType.systemProperty(cleanerType)) diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index a25b37a2a9..1407c39bfb 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -30,7 +30,7 @@ import java.lang.management.ManagementFactory import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.ints.IntOpenHashSet -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, SparkContext, Logging} /** * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in @@ -41,6 +41,7 @@ import org.apache.spark.Logging */ private[spark] object SizeEstimator extends Logging { + private def conf = SparkContext.globalConf // Sizes of primitive types private val BYTE_SIZE = 1 private val BOOLEAN_SIZE = 1 @@ -90,8 +91,8 @@ private[spark] object SizeEstimator extends Logging { } private def getIsCompressedOops : Boolean = { - if (System.getProperty("spark.test.useCompressedOops") != null) { - return System.getProperty("spark.test.useCompressedOops").toBoolean + if (conf.getOrElse("spark.test.useCompressedOops", null) != null) { + return conf.get("spark.test.useCompressedOops").toBoolean } try { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 3f7858d2de..fd5888e525 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -36,7 +36,7 @@ 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.{SparkException, Logging} +import org.apache.spark.{SparkContext, SparkException, Logging} /** @@ -44,6 +44,7 @@ import org.apache.spark.{SparkException, Logging} */ private[spark] object Utils extends Logging { + private lazy val conf = SparkContext.globalConf /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -312,7 +313,7 @@ private[spark] object Utils extends Logging { * multiple paths. */ def getLocalDir: String = { - System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) + conf.getOrElse("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) } /** @@ -398,7 +399,7 @@ private[spark] object Utils extends Logging { } def localHostPort(): String = { - val retval = System.getProperty("spark.hostPort", null) + val retval = conf.getOrElse("spark.hostPort", null) if (retval == null) { logErrorWithStack("spark.hostPort not set but invoking localHostPort") return localHostName() diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 271dc905bc..10b8b441fd 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AkkaUtils class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { - + private val conf = new SparkConf test("compressSize") { assert(MapOutputTracker.compressSize(0L) === 0) assert(MapOutputTracker.compressSize(1L) === 1) @@ -48,14 +48,14 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master start and stop") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTrackerMaster() + val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) tracker.stop() } test("master register and fetch") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTrackerMaster() + val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) @@ -74,7 +74,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master register and unregister and fetch") { val actorSystem = ActorSystem("test") - val tracker = new MapOutputTrackerMaster() + val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) @@ -96,16 +96,16 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("remote fetch") { val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) - val masterTracker = new MapOutputTrackerMaster() + val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = Left(actorSystem.actorOf( Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker")) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0) - val slaveTracker = new MapOutputTracker() + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf) + val slaveTracker = new MapOutputTracker(conf) slaveTracker.trackerActor = Right(slaveSystem.actorSelection( "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker")) diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index 7181333adf..4ecdde0001 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -19,17 +19,19 @@ package org.apache.spark.metrics import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.deploy.master.MasterSource +import org.apache.spark.SparkConf class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ - + var conf: SparkConf = null before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() System.setProperty("spark.metrics.conf", filePath) + conf = new SparkConf } test("MetricsSystem with default config") { - val metricsSystem = MetricsSystem.createMetricsSystem("default") + val metricsSystem = MetricsSystem.createMetricsSystem("default", conf) val sources = metricsSystem.sources val sinks = metricsSystem.sinks @@ -39,7 +41,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { } test("MetricsSystem with sources add") { - val metricsSystem = MetricsSystem.createMetricsSystem("test") + val metricsSystem = MetricsSystem.createMetricsSystem("test", conf) val sources = metricsSystem.sources val sinks = metricsSystem.sinks 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 706d84a58b..2aa259daf3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -17,21 +17,14 @@ package org.apache.spark.scheduler -import scala.collection.mutable.{Map, HashMap} - -import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter - -import org.apache.spark.LocalSparkContext -import org.apache.spark.MapOutputTrackerMaster -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 scala.Tuple2 +import scala.collection.mutable.{HashMap, Map} + +import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} +import org.scalatest.{BeforeAndAfter, FunSuite} /** * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler @@ -46,7 +39,7 @@ import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} * and capturing the resulting TaskSets from the mock TaskScheduler. */ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { - + val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() val taskScheduler = new TaskScheduler() { @@ -74,7 +67,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont */ val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]] // stub out BlockManagerMaster.getLocations to use our cacheLocations - val blockManagerMaster = new BlockManagerMaster(null) { + val blockManagerMaster = new BlockManagerMaster(null, conf) { override def getLocations(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = { blockIds.map { _.asRDDId.map(id => (id.rddId -> id.splitIndex)).flatMap(key => cacheLocations.get(key)). @@ -99,7 +92,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont taskSets.clear() cacheLocations.clear() results.clear() - mapOutputTracker = new MapOutputTrackerMaster() + mapOutputTracker = new MapOutputTrackerMaster(conf) scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, sc.env) { override def runLocally(job: ActiveJob) { // don't bother with the thread while unit testing 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 002368ff55..dd122615ad 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -95,7 +95,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) - val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) + val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) joblogger.getLogDir should be ("/tmp/spark-%s".format(user)) joblogger.getJobIDtoPrintWriter.size should be (1) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala index 95d3553d91..34d2e4cb8c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala @@ -169,7 +169,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() System.setProperty("spark.scheduler.allocation.file", xmlPath) val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) - val schedulableBuilder = new FairSchedulableBuilder(rootPool) + val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) schedulableBuilder.buildPools() assert(rootPool.getSchedulableByName("default") != null) 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 bb28a31a99..2bb827c022 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 @@ -81,8 +81,8 @@ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* 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 + private val conf = new SparkConf + val LOCALITY_WAIT = conf.getOrElse("spark.locality.wait", "3000").toLong test("TaskSet with no preferences") { sc = new SparkContext("local", "test") diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala index 27c2d53361..618fae7c16 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} -import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv} +import org.apache.spark.{SparkConf, LocalSparkContext, SparkContext, SparkEnv} import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, TaskResult} import org.apache.spark.storage.TaskResultBlockId 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 5b4d63b954..4ef5538951 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -31,8 +31,10 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.util.{SizeEstimator, Utils, AkkaUtils, ByteBufferInputStream} import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.{SparkConf, SparkContext} class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { + private val conf = new SparkConf var store: BlockManager = null var store2: BlockManager = null var actorSystem: ActorSystem = null @@ -42,7 +44,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT var oldHeartBeat: String = null // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test - System.setProperty("spark.kryoserializer.buffer.mb", "1") + conf.set("spark.kryoserializer.buffer.mb", "1") val serializer = new KryoSerializer // Implicitly convert strings to BlockIds for test clarity. @@ -50,22 +52,23 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) before { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0, conf = conf) this.actorSystem = actorSystem - System.setProperty("spark.driver.port", boundPort.toString) - System.setProperty("spark.hostPort", "localhost:" + boundPort) + conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.hostPort", "localhost:" + boundPort) master = new BlockManagerMaster( - Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))) + Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf) // 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") - oldHeartBeat = System.setProperty("spark.storage.disableBlockManagerHeartBeat", "true") + System.setProperty("os.arch", "amd64") + conf.set("os.arch", "amd64") + conf.set("spark.test.useCompressedOops", "true") + conf.set("spark.storage.disableBlockManagerHeartBeat", "true") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() // Set some value ... - System.setProperty("spark.hostPort", Utils.localHostName() + ":" + 1111) + conf.set("spark.hostPort", Utils.localHostName() + ":" + 1111) } after { @@ -86,13 +89,13 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT master = null if (oldArch != null) { - System.setProperty("os.arch", oldArch) + conf.set("os.arch", oldArch) } else { System.clearProperty("os.arch") } if (oldOops != null) { - System.setProperty("spark.test.useCompressedOops", oldOops) + conf.set("spark.test.useCompressedOops", oldOops) } else { System.clearProperty("spark.test.useCompressedOops") } @@ -133,7 +136,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 1 manager interaction") { - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -163,8 +166,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 2 managers interaction") { - store = new BlockManager("exec1", actorSystem, master, serializer, 2000) - store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer, 2000) + store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf) + store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer, 2000, conf) val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -179,7 +182,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing block") { - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -227,7 +230,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing rdd") { - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -261,7 +264,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("reregistration on heart beat") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -277,7 +280,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("reregistration on block update") { - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) @@ -296,7 +299,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("reregistration doesn't dead lock") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -333,7 +336,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -352,7 +355,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -371,7 +374,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of same RDD") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -390,7 +393,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of multiple RDDs") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) @@ -413,7 +416,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("on-disk storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -426,7 +429,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -441,7 +444,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -456,7 +459,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -471,7 +474,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization and getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -486,7 +489,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -511,7 +514,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU with streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -535,7 +538,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels and streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -581,7 +584,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("overly large block") { - store = new BlockManager("", actorSystem, master, serializer, 500) + store = new BlockManager("", actorSystem, master, serializer, 500, conf) store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK) @@ -591,53 +594,53 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block compression") { try { - System.setProperty("spark.shuffle.compress", "true") - store = new BlockManager("exec1", actorSystem, master, serializer, 2000) + conf.set("spark.shuffle.compress", "true") + store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") store.stop() store = null - System.setProperty("spark.shuffle.compress", "false") - store = new BlockManager("exec2", actorSystem, master, serializer, 2000) + conf.set("spark.shuffle.compress", "false") + store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 1000, "shuffle_0_0_0 was compressed") store.stop() store = null - System.setProperty("spark.broadcast.compress", "true") - store = new BlockManager("exec3", actorSystem, master, serializer, 2000) + conf.set("spark.broadcast.compress", "true") + store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf) store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 100, "broadcast_0 was not compressed") store.stop() store = null - System.setProperty("spark.broadcast.compress", "false") - store = new BlockManager("exec4", actorSystem, master, serializer, 2000) + conf.set("spark.broadcast.compress", "false") + store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf) store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 1000, "broadcast_0 was compressed") store.stop() store = null - System.setProperty("spark.rdd.compress", "true") - store = new BlockManager("exec5", actorSystem, master, serializer, 2000) + conf.set("spark.rdd.compress", "true") + store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf) store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) <= 100, "rdd_0_0 was not compressed") store.stop() store = null - System.setProperty("spark.rdd.compress", "false") - store = new BlockManager("exec6", actorSystem, master, serializer, 2000) + conf.set("spark.rdd.compress", "false") + store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf) store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) >= 1000, "rdd_0_0 was compressed") store.stop() store = null // Check that any other block types are also kept uncompressed - store = new BlockManager("exec7", actorSystem, master, serializer, 2000) + store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf) store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed") store.stop() @@ -651,7 +654,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block store put failure") { // Use Java serializer so we can create an unserializable error. - store = new BlockManager("", actorSystem, master, new JavaSerializer, 1200) + store = new BlockManager("", actorSystem, master, new JavaSerializer, 1200, conf) // The put should fail since a1 is not serializable. class UnserializableClass diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 070982e798..f940448abd 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -17,15 +17,18 @@ package org.apache.spark.storage -import java.io.{FileWriter, File} +import java.io.{File, FileWriter} import scala.collection.mutable import com.google.common.io.Files +import org.apache.spark.SparkConf import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} +import scala.util.Try +import akka.actor.{Props, ActorSelection, ActorSystem} class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { - + private val testConf = new SparkConf val rootDir0 = Files.createTempDir() rootDir0.deleteOnExit() val rootDir1 = Files.createTempDir() @@ -36,10 +39,11 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before // This suite focuses primarily on consolidation features, // so we coerce consolidation if not already enabled. val consolidateProp = "spark.shuffle.consolidateFiles" - val oldConsolidate = Option(System.getProperty(consolidateProp)) - System.setProperty(consolidateProp, "true") + val oldConsolidate = Try(testConf.get(consolidateProp)).toOption + testConf.set(consolidateProp, "true") val shuffleBlockManager = new ShuffleBlockManager(null) { + override def conf = testConf.clone var idToSegmentMap = mutable.Map[ShuffleBlockId, FileSegment]() override def getBlockLocation(id: ShuffleBlockId) = idToSegmentMap(id) } diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index 5aff26f9fc..a5facd5bbd 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.util import org.scalatest.FunSuite import org.scalatest.BeforeAndAfterAll import org.scalatest.PrivateMethodTester +import org.apache.spark.SparkContext class DummyClass1 {} @@ -139,7 +140,8 @@ class SizeEstimatorSuite test("64-bit arch with no compressed oops") { val arch = System.setProperty("os.arch", "amd64") val oops = System.setProperty("spark.test.useCompressedOops", "false") - + SparkContext.globalConf.set("os.arch", "amd64") + SparkContext.globalConf.set("spark.test.useCompressedOops", "false") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() 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 72b5c7b88e..12c430be27 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 @@ -36,16 +36,18 @@ object WikipediaPageRank { System.err.println("Usage: WikipediaPageRank ") System.exit(-1) } - - System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - System.setProperty("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) + val sparkConf = new SparkConf() + sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + sparkConf.set("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) val inputFile = args(0) val threshold = args(1).toDouble val numPartitions = args(2).toInt val host = args(3) val usePartitioner = args(4).toBoolean - val sc = new SparkContext(host, "WikipediaPageRank") + + sparkConf.setMasterUrl(host).setAppName("WikipediaPageRank") + val sc = new SparkContext(sparkConf) // Parse the Wikipedia page data into a graph val input = sc.textFile(inputFile) 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 ddf6855325..5bf0b7a24a 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 @@ -34,15 +34,19 @@ object WikipediaPageRankStandalone { System.err.println("Usage: WikipediaPageRankStandalone ") System.exit(-1) } + val sparkConf = new SparkConf() + sparkConf.set("spark.serializer", "spark.bagel.examples.WPRSerializer") - System.setProperty("spark.serializer", "spark.bagel.examples.WPRSerializer") val inputFile = args(0) val threshold = args(1).toDouble val numIterations = args(2).toInt val host = args(3) val usePartitioner = args(4).toBoolean - val sc = new SparkContext(host, "WikipediaPageRankStandalone") + + sparkConf.setMasterUrl(host).setAppName("WikipediaPageRankStandalone") + + val sc = new SparkContext(sparkConf) val input = sc.textFile(inputFile) val partitioner = new HashPartitioner(sc.defaultParallelism) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 50e3f9639c..2402409e6e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -26,6 +26,7 @@ import akka.actor.ActorRef import akka.actor.Props import akka.actor.actorRef2Scala +import org.apache.spark.SparkConf import org.apache.spark.streaming.Seconds import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions @@ -116,7 +117,7 @@ object FeederActor { val Seq(host, port) = args.toSeq - val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt)._1 + val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt, conf = new SparkConf)._1 val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor") println("Feeder started as:" + feeder) 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 36853acab5..2f2d106f86 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 @@ -578,14 +578,13 @@ object ALS { val implicitPrefs = if (args.length >= 7) args(6).toBoolean else false val alpha = if (args.length >= 8) args(7).toDouble else 1 val blocks = if (args.length == 9) args(8).toInt else -1 - - 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") - System.setProperty("spark.locality.wait", "10000") - val sc = new SparkContext(master, "ALS") + sc.conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + sc.conf.set("spark.kryo.registrator", classOf[ALSRegistrator].getName) + sc.conf.set("spark.kryo.referenceTracking", "false") + sc.conf.set("spark.kryoserializer.buffer.mb", "8") + sc.conf.set("spark.locality.wait", "10000") + val ratings = sc.textFile(ratingsFile).map { line => val fields = line.split(',') Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index eeeca3ea8a..433268a1dd 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -61,13 +61,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private var amClient: AMRMClient[ContainerRequest] = _ // Default to numWorkers * 2, with minimum of 3 - private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures", + private val maxNumWorkerFailures = conf.getOrElse("spark.yarn.max.worker.failures", math.max(args.numWorkers * 2, 3).toString()).toInt def run() { // Setup the directories so things go to YARN approved directories rather // than user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) + conf.set("spark.local.dir", getLocalDirs()) // Use priority 30 as it's higher then HDFS. It's same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) @@ -138,10 +138,10 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("Waiting for Spark driver to be reachable.") var driverUp = false var tries = 0 - val numTries = System.getProperty("spark.yarn.applicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt while (!driverUp && tries < numTries) { - val driverHost = System.getProperty("spark.driver.host") - val driverPort = System.getProperty("spark.driver.port") + val driverHost = conf.get("spark.driver.host") + val driverPort = conf.get("spark.driver.port") try { val socket = new Socket(driverHost, driverPort.toInt) socket.close() @@ -199,7 +199,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e ApplicationMaster.sparkContextRef.synchronized { var numTries = 0 val waitTime = 10000L - val maxNumTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt + val maxNumTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries) { logInfo("Waiting for Spark context initialization ... " + numTries) numTries = numTries + 1 @@ -265,7 +265,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) @@ -343,7 +343,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private def cleanupStagingDir() { var stagingDirPath: Path = null try { - val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 94678815e8..a322f60864 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -244,7 +244,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort + val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort if (UserGroupInformation.isSecurityEnabled()) { val dstFs = dst.getFileSystem(conf) @@ -499,7 +499,7 @@ object Client { Path.SEPARATOR + LOG4J_PROP) } // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false") + val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") .toBoolean if (userClasspathFirst) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 70be15d0a3..41ac292249 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -35,7 +35,7 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 // MB var workerCores = 1 var numWorkers = 2 - var amQueue = System.getProperty("QUEUE", "default") + var amQueue = conf.getOrElse("QUEUE", "default") var amMemory: Int = 512 // MB var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index bc31bb2eb0..f7d73f0d83 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -136,8 +136,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte Thread.sleep(100) } } - System.setProperty("spark.driver.host", driverHost) - System.setProperty("spark.driver.port", driverPort.toString) + conf.set("spark.driver.host", driverHost) + conf.set("spark.driver.port", driverPort.toString) val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index c27257cda4..71d1cbd416 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -254,8 +254,8 @@ private[yarn] class YarnAllocationHandler( } else { val workerId = workerIdCounter.incrementAndGet().toString val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), - System.getProperty("spark.driver.port"), + conf.get("spark.driver.host"), + conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("Launching container %s for on host %s".format(containerId, workerHostname)) diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index b206780c78..6feaaff014 100644 --- a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -52,8 +52,8 @@ private[spark] class YarnClientSchedulerBackend( if (workerNumber == null) workerNumber = defaultWorkerNumber - val driverHost = System.getProperty("spark.driver.host") - val driverPort = System.getProperty("spark.driver.port") + val driverHost = conf.get("spark.driver.host") + val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort val argsArray = Array[String]( diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ab96cfa18b..ffb54a24ac 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -244,7 +244,8 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-ganglia" % "3.0.0", "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1", - "com.twitter" % "chill-java" % "0.3.1" + "com.twitter" % "chill-java" % "0.3.1", + "com.typesafe" % "config" % "1.0.2" ) ) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 523fd1222d..b2f499e637 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -930,9 +930,6 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, def createSparkContext(): SparkContext = { val uri = System.getenv("SPARK_EXECUTOR_URI") - if (uri != null) { - System.setProperty("spark.executor.uri", uri) - } val master = this.master match { case Some(m) => m case None => { @@ -942,6 +939,10 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } val jars = SparkILoop.getAddedJars.map(new java.io.File(_).getAbsolutePath) sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars) + if (uri != null) { + sparkContext.conf.set("spark.executor.uri", uri) + } + sparkContext.conf.set("spark.repl.class.uri", intp.classServer.uri) echo("Created spark context..") sparkContext } 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 e1455ef8a1..0d412e4478 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -34,10 +34,8 @@ import scala.tools.reflect.StdRuntimeTags._ import scala.util.control.ControlThrowable import util.stackTraceString -import org.apache.spark.HttpServer +import org.apache.spark.{SparkContext, HttpServer, SparkEnv, Logging} import org.apache.spark.util.Utils -import org.apache.spark.SparkEnv -import org.apache.spark.Logging // /** directory to save .class files to */ // private class ReplVirtualDirectory(out: JPrintWriter) extends VirtualDirectory("((memory))", None) { @@ -91,7 +89,7 @@ import org.apache.spark.Logging /** Local directory to save .class files too */ val outputDir = { val tmp = System.getProperty("java.io.tmpdir") - val rootDir = System.getProperty("spark.repl.classdir", tmp) + val rootDir = SparkContext.globalConf.getOrElse("spark.repl.classdir", tmp) Utils.createTempDir(rootDir) } if (SPARK_DEBUG_REPL) { @@ -112,7 +110,6 @@ import org.apache.spark.Logging // 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) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 9271914eb5..b8e1427a21 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -34,7 +34,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) extends Logging with Serializable { val master = ssc.sc.master val framework = ssc.sc.appName - val sparkHome = ssc.sc.sparkHome + val sparkHome = ssc.sc.getSparkHome.getOrElse(null) val jars = ssc.sc.jars val environment = ssc.sc.environment val graph = ssc.graph @@ -42,6 +42,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) val checkpointDuration = ssc.checkpointDuration val pendingTimes = ssc.scheduler.jobManager.getPendingTimes() val delaySeconds = MetadataCleaner.getDelaySeconds + val sparkConf = ssc.sc.conf def validate() { assert(master != null, "Checkpoint.master is null") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala index ed892e33e6..1d23713c80 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala @@ -26,7 +26,7 @@ class Scheduler(ssc: StreamingContext) extends Logging { initLogging() - val concurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt + val concurrentJobs = ssc.sc.conf.getOrElse("spark.streaming.concurrentJobs", "1").toInt val jobManager = new JobManager(ssc, concurrentJobs) val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) { new CheckpointWriter(ssc.checkpointDir) @@ -34,7 +34,7 @@ class Scheduler(ssc: StreamingContext) extends Logging { null } - val clockClass = System.getProperty( + val clockClass = ssc.sc.conf.getOrElse( "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, @@ -73,7 +73,7 @@ class Scheduler(ssc: StreamingContext) extends Logging { // or if the property is defined set it to that time if (clock.isInstanceOf[ManualClock]) { val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds - val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong + val jumpTime = ssc.sc.conf.getOrElse("spark.streaming.manualClock.jump", "0").toLong clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime) } 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 d2c4fdee65..76744223e1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -115,7 +115,7 @@ class StreamingContext private ( protected[streaming] val sc: SparkContext = { if (isCheckpointPresent) { - new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars, cp_.environment) + new SparkContext(cp_.sparkConf, cp_.environment) } else { sc_ } 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 d5ae8aef92..8bf761b8cb 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 @@ -175,8 +175,8 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging /** A helper actor that communicates with the NetworkInputTracker */ private class NetworkReceiverActor extends Actor { logInfo("Attempting to register with tracker") - val ip = System.getProperty("spark.driver.host", "localhost") - val port = System.getProperty("spark.driver.port", "7077").toInt + val ip = env.conf.getOrElse("spark.driver.host", "localhost") + val port = env.conf.getOrElse("spark.driver.port", "7077").toInt val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) val tracker = env.actorSystem.actorSelection(url) val timeout = 5.seconds @@ -213,7 +213,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null) val clock = new SystemClock() - val blockInterval = System.getProperty("spark.streaming.blockInterval", "200").toLong + val blockInterval = env.conf.getOrElse("spark.streaming.blockInterval", "200").toLong val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer) val blockStorageLevel = storageLevel val blocksForPushing = new ArrayBlockingQueue[Block](1000) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index e81287b44e..315bd5443c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -42,7 +42,7 @@ import org.apache.spark.streaming.util.ManualClock */ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") before { FileUtils.deleteDirectory(new File(checkpointDir)) @@ -69,7 +69,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second") - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") val stateStreamCheckpointInterval = Seconds(1) @@ -135,13 +135,13 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // Restart stream computation from the new checkpoint file to see whether that file has // correct checkpoint data + conf.set("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString) ssc = new StreamingContext(checkpointDir) stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head logInfo("Restored data of state stream = \n[" + stateStream.generatedRDDs.mkString("\n") + "]") assert(!stateStream.generatedRDDs.isEmpty, "No restored RDDs in state stream after recovery from second failure") // Adjust manual clock time as if it is being restarted after a delay - System.setProperty("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString) ssc.start() advanceTimeWithRealDelay(ssc, 4) ssc.stop() diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 7dc82decef..da8f135dd7 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -53,7 +53,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { override def checkpointDir = "checkpoint" before { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } after { @@ -68,7 +68,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { testServer.start() // Set up the streaming context and input streams - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String ]] val outputStream = new TestOutputStream(networkStream, outputBuffer) @@ -113,7 +113,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { test("flume input stream") { // Set up the streaming context and input streams - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] @@ -162,11 +162,11 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { test("file input stream") { // Disable manual clock as FileInputDStream does not work with manual clock - System.clearProperty("spark.streaming.clock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") // Set up the streaming context and input streams val testDir = Files.createTempDir() - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val fileStream = ssc.textFileStream(testDir.toString) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] def output = outputBuffer.flatMap(x => x) @@ -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", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } @@ -218,7 +218,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { testServer.start() // Set up the streaming context and input streams - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val networkStream = ssc.actorStream[String](Props(new TestActor(port)), "TestActor", StorageLevel.MEMORY_AND_DISK) //Had to pass the local value of port to prevent from closing over entire scope val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] @@ -262,7 +262,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } test("kafka input stream") { - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val topics = Map("my-topic" -> 1) val test1 = ssc.kafkaStream("localhost:12345", "group", topics) val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK) @@ -285,7 +285,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { MultiThreadTestReceiver.haveAllThreadsFinished = false // set up the network stream using the test receiver - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val networkStream = ssc.networkStream[Int](testReceiver) val countStream = networkStream.count val outputBuffer = new ArrayBuffer[Seq[Long]] with SynchronizedBuffer[Seq[Long]] 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 2f34e812a1..d1cab0c609 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -28,7 +28,7 @@ import java.io.{ObjectInputStream, IOException} import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, SparkConf, Logging} import org.apache.spark.rdd.RDD /** @@ -130,6 +130,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Whether to actually wait in real time before changing manual clock def actuallyWait = false + def conf = new SparkConf().setMasterUrl(master).setAppName(framework).set("spark.cleaner.ttl", "3600") /** * Set up required DStreams to test the DStream operation using the two sequences * of input collections. @@ -139,9 +140,9 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { operation: DStream[U] => DStream[V], numPartitions: Int = numInputPartitions ): StreamingContext = { - + val sc = new SparkContext(conf) // Create StreamingContext - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(sc, batchDuration) if (checkpointDir != null) { ssc.checkpoint(checkpointDir) } @@ -165,9 +166,9 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { input2: Seq[Seq[V]], operation: (DStream[U], DStream[V]) => DStream[W] ): StreamingContext = { - + val sc = new SparkContext(conf) // Create StreamingContext - val ssc = new StreamingContext(master, framework, batchDuration) + val ssc = new StreamingContext(sc, batchDuration) if (checkpointDir != null) { ssc.checkpoint(checkpointDir) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 240ed8b32a..1dd38dd13e 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -58,13 +58,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) private var isLastAMRetry: Boolean = true // default to numWorkers * 2, with minimum of 3 - private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures", + private val maxNumWorkerFailures = conf.getOrElse("spark.yarn.max.worker.failures", math.max(args.numWorkers * 2, 3).toString()).toInt def run() { // Setup the directories so things go to yarn approved directories rather // then user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) + conf.set("spark.local.dir", getLocalDirs()) // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) @@ -165,10 +165,10 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("Waiting for spark driver to be reachable.") var driverUp = false var tries = 0 - val numTries = System.getProperty("spark.yarn.applicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt while(!driverUp && tries < numTries) { - val driverHost = System.getProperty("spark.driver.host") - val driverPort = System.getProperty("spark.driver.port") + val driverHost = conf.get("spark.driver.host") + val driverPort = conf.get("spark.driver.port") try { val socket = new Socket(driverHost, driverPort.toInt) socket.close() @@ -226,7 +226,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e ApplicationMaster.sparkContextRef.synchronized { var count = 0 val waitTime = 10000L - val numTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt while (ApplicationMaster.sparkContextRef.get() == null && count < numTries) { logInfo("Waiting for spark context initialization ... " + count) count = count + 1 @@ -294,7 +294,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) @@ -377,7 +377,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private def cleanupStagingDir() { var stagingDirPath: Path = null try { - val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 79dd038065..29892e98e3 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -230,7 +230,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort + val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort if (UserGroupInformation.isSecurityEnabled()) { val dstFs = dst.getFileSystem(conf) @@ -461,7 +461,7 @@ 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") + conf.set("SPARK_YARN_MODE", "true") val args = new ClientArguments(argStrings) @@ -483,7 +483,7 @@ object Client { Path.SEPARATOR + LOG4J_PROP) } // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false") + val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") .toBoolean if (userClasspathFirst) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index b3a7886d93..617289f568 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -33,7 +33,7 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 var workerCores = 1 var numWorkers = 2 - var amQueue = System.getProperty("QUEUE", "default") + var amQueue = conf.getOrElse("QUEUE", "default") var amMemory: Int = 512 var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 69038844bb..c1e79cbe66 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -162,8 +162,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte Thread.sleep(100) } } - System.setProperty("spark.driver.host", driverHost) - System.setProperty("spark.driver.port", driverPort.toString) + conf.set("spark.driver.host", driverHost) + conf.set("spark.driver.port", driverPort.toString) val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 9ab2073529..4c9fee5695 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -239,7 +239,7 @@ private[yarn] class YarnAllocationHandler( // (workerIdCounter) val workerId = workerIdCounter.incrementAndGet().toString val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), + conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("launching container on " + containerId + " host " + workerHostname) diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index b206780c78..6feaaff014 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -52,8 +52,8 @@ private[spark] class YarnClientSchedulerBackend( if (workerNumber == null) workerNumber = defaultWorkerNumber - val driverHost = System.getProperty("spark.driver.host") - val driverPort = System.getProperty("spark.driver.port") + val driverHost = conf.get("spark.driver.host") + val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort val argsArray = Array[String]( -- cgit v1.2.3 From d4dfab503a9222b5acf5c4bf69b91c16f298e4aa Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 24 Dec 2013 14:01:13 -0800 Subject: Fixed Python API for sc.setCheckpointDir. Also other fixes based on Reynold's comments on PR 289. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 ++-- core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala | 2 -- .../main/scala/org/apache/spark/rdd/RDDCheckpointData.scala | 2 +- python/pyspark/context.py | 9 ++------- python/pyspark/tests.py | 4 ++-- .../org/apache/spark/streaming/dstream/FileInputDStream.scala | 6 +++--- .../org/apache/spark/streaming/scheduler/JobGenerator.scala | 11 ++++++----- 7 files changed, 16 insertions(+), 22 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c30f896cf1..cc87febf33 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -861,12 +861,12 @@ class SparkContext( * be a HDFS path if running on a cluster. */ def setCheckpointDir(directory: String) { - checkpointDir = Option(directory).map(dir => { + checkpointDir = Option(directory).map { dir => val path = new Path(dir, UUID.randomUUID().toString) val fs = path.getFileSystem(hadoopConfiguration) fs.mkdirs(path) fs.getFileStatus(path).getPath().toString - }) + } } /** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */ diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 80385fce57..293a7d1f68 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -18,9 +18,7 @@ package org.apache.spark.rdd import java.io.IOException - import scala.reflect.ClassTag -import java.io.{IOException} import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index 5a565d7e78..091a6fdb54 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -95,7 +95,7 @@ private[spark] class RDDCheckpointData[T: ClassTag](rdd: RDD[T]) rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path.toString, broadcastedConf) _) val newRDD = new CheckpointRDD[T](rdd.context, path.toString) if (newRDD.partitions.size != rdd.partitions.size) { - throw new Exception( + throw new SparkException( "Checkpoint RDD " + newRDD + "("+ newRDD.partitions.size + ") has different " + "number of partitions than original RDD " + rdd + "(" + rdd.partitions.size + ")") } diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 0604f6836c..108f36576a 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -320,17 +320,12 @@ class SparkContext(object): 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): + def setCheckpointDir(self, dirName): """ Set the directory under which RDDs are going to be checkpointed. The directory must be a HDFS path if running on a cluster. - - If the directory does not exist, it will be created. If the directory - exists and C{useExisting} is set to true, then the exisiting directory - will be used. Otherwise an exception will be thrown to prevent - accidental overriding of checkpoint files in the existing directory. """ - self._jsc.sc().setCheckpointDir(dirName, useExisting) + self._jsc.sc().setCheckpointDir(dirName) def _getJavaStorageLevel(self, storageLevel): """ diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 3987642bf4..7acb6eaf10 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -73,8 +73,8 @@ class TestCheckpoint(PySparkTestCase): time.sleep(1) # 1 second self.assertTrue(flatMappedRDD.isCheckpointed()) self.assertEqual(flatMappedRDD.collect(), result) - self.assertEqual(self.checkpointDir.name, - os.path.dirname(flatMappedRDD.getCheckpointFile())) + self.assertEqual("file:" + self.checkpointDir.name, + os.path.dirname(os.path.dirname(flatMappedRDD.getCheckpointFile()))) def test_checkpoint_and_restore(self): parCollection = self.sc.parallelize([1, 2, 3, 4]) 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 4a7c5cf29c..d6514a1fb1 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 @@ -123,7 +123,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas reset() } } - (Seq(), -1, Seq()) + (Seq.empty, -1, Seq.empty) } /** Generate one RDD from an array of files */ @@ -193,7 +193,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas * been seen before (i.e. the file should not be in lastModTimeFiles) */ private[streaming] - class CustomPathFilter(currentTime: Long) extends PathFilter() { + class CustomPathFilter(currentTime: Long) extends PathFilter { // Latest file mod time seen in this round of fetching files and its corresponding files var latestModTime = 0L val latestModTimeFiles = new HashSet[String]() @@ -209,7 +209,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas logDebug("Rejected by filter " + path) return false } else { // Accept file only if - val modTime = fs.getFileStatus(path).getModificationTime() + val modTime = fs.getFileStatus(path).getModificationTime() logDebug("Mod time for " + path + " is " + modTime) if (modTime < prevModTime) { logDebug("Mod time less than last mod time") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 2552d51654..921a33a4cb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -17,16 +17,17 @@ package org.apache.spark.streaming.scheduler +import akka.actor.{Props, Actor} import org.apache.spark.SparkEnv import org.apache.spark.Logging import org.apache.spark.streaming.{Checkpoint, Time, CheckpointWriter} import org.apache.spark.streaming.util.{ManualClock, RecurringTimer, Clock} -import akka.actor.{Props, Actor} -sealed trait JobGeneratorEvent -case class GenerateJobs(time: Time) extends JobGeneratorEvent -case class ClearOldMetadata(time: Time) extends JobGeneratorEvent -case class DoCheckpoint(time: Time) extends JobGeneratorEvent +/** Event classes for JobGenerator */ +private[scheduler] sealed trait JobGeneratorEvent +private[scheduler] case class GenerateJobs(time: Time) extends JobGeneratorEvent +private[scheduler] case class ClearOldMetadata(time: Time) extends JobGeneratorEvent +private[scheduler] case class DoCheckpoint(time: Time) extends JobGeneratorEvent /** * This class generates jobs from DStreams as well as drives checkpointing and cleaning -- cgit v1.2.3 From 1efe3adf560d207f9106ffd4e15934e422adb636 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 24 Dec 2013 14:18:39 -0800 Subject: Responded to Reynold's style comments --- .../main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 7 ++++--- .../src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 4 ++-- .../main/scala/org/apache/spark/scheduler/local/LocalBackend.scala | 2 +- 3 files changed, 7 insertions(+), 6 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 7409168f7b..dbac6b96ac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -46,9 +46,10 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode * we are holding a lock on ourselves. */ private[spark] class TaskSchedulerImpl( - val sc: SparkContext, - val maxTaskFailures : Int = System.getProperty("spark.task.maxFailures", "4").toInt, - isLocal: Boolean = false) extends TaskScheduler with Logging { + val sc: SparkContext, + val maxTaskFailures : Int = System.getProperty("spark.task.maxFailures", "4").toInt, + isLocal: Boolean = false) + extends TaskScheduler with Logging { // How often to check for speculative tasks val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index aa3fb0b35a..c676e73e03 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -17,6 +17,7 @@ package org.apache.spark.scheduler +import java.io.NotSerializableException import java.util.Arrays import scala.collection.mutable.ArrayBuffer @@ -28,8 +29,7 @@ import scala.math.min import org.apache.spark.{ExceptionFailure, FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskKilled, TaskResultLost, TaskState} import org.apache.spark.TaskState.TaskState -import org.apache.spark.util.{SystemClock, Clock} -import java.io.NotSerializableException +import org.apache.spark.util.{Clock, SystemClock} /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 69c1c04843..4edc6a0d3f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -93,7 +93,7 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: } override def reviveOffers() { - localActor ! ReviveOffers + localActor ! ReviveOffers } override def defaultParallelism() = totalCores -- cgit v1.2.3 From 040dd3ecd5a7668ecbb6cf0b611318c1010f9bdb Mon Sep 17 00:00:00 2001 From: Binh Nguyen Date: Sat, 7 Dec 2013 00:44:10 -0800 Subject: upgrade Netty from 4.0.0.Beta2 to 4.0.13.Final --- .../org/apache/spark/network/netty/FileClient.java | 15 +++++++---- .../netty/FileClientChannelInitializer.java | 5 ++-- .../spark/network/netty/FileClientHandler.java | 12 +++------ .../org/apache/spark/network/netty/FileServer.java | 29 ++++++++++++++++------ .../spark/network/netty/FileServerHandler.java | 8 +++--- pom.xml | 2 +- project/SparkBuild.scala | 2 +- 7 files changed, 42 insertions(+), 31 deletions(-) (limited to 'core/src') diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java index edd0fc56f8..95e25d94e6 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClient.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClient.java @@ -23,6 +23,7 @@ import io.netty.channel.ChannelOption; import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.oio.OioSocketChannel; +import io.netty.util.concurrent.EventExecutorGroup; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,6 +33,7 @@ class FileClient { private FileClientHandler handler = null; private Channel channel = null; private Bootstrap bootstrap = null; + private EventLoopGroup group = null; private int connectTimeout = 60*1000; // 1 min public FileClient(FileClientHandler handler, int connectTimeout) { @@ -40,8 +42,9 @@ class FileClient { } public void init() { - bootstrap = new Bootstrap(); - bootstrap.group(new OioEventLoopGroup()) + group = new OioEventLoopGroup(); + Bootstrap bootstrap = new Bootstrap(); + bootstrap.group(group) .channel(OioSocketChannel.class) .option(ChannelOption.SO_KEEPALIVE, true) .option(ChannelOption.TCP_NODELAY, true) @@ -76,11 +79,13 @@ class FileClient { public void close() { if(channel != null) { - channel.close(); + channel.close().awaitUninterruptibly(); channel = null; } - if ( bootstrap!=null) { - bootstrap.shutdown(); + + if (group!=null) { + group.shutdownGracefully(); + group = null; bootstrap = null; } } diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java index 65ee15d63b..b16225026b 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java @@ -17,7 +17,6 @@ package org.apache.spark.network.netty; -import io.netty.buffer.BufType; import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; import io.netty.handler.codec.string.StringEncoder; @@ -25,7 +24,7 @@ import io.netty.handler.codec.string.StringEncoder; class FileClientChannelInitializer extends ChannelInitializer { - private FileClientHandler fhandler; + private final FileClientHandler fhandler; public FileClientChannelInitializer(FileClientHandler handler) { fhandler = handler; @@ -35,7 +34,7 @@ class FileClientChannelInitializer extends ChannelInitializer { public void initChannel(SocketChannel channel) { // file no more than 2G channel.pipeline() - .addLast("encoder", new StringEncoder(BufType.BYTE)) + .addLast("encoder", new StringEncoder()) .addLast("handler", fhandler); } } diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java index 8a09210245..63d3d92725 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java @@ -19,11 +19,11 @@ package org.apache.spark.network.netty; import io.netty.buffer.ByteBuf; import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelInboundByteHandlerAdapter; +import io.netty.channel.SimpleChannelInboundHandler; import org.apache.spark.storage.BlockId; -abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter { +abstract class FileClientHandler extends SimpleChannelInboundHandler { private FileHeader currentHeader = null; @@ -37,13 +37,7 @@ abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter { public abstract void handleError(BlockId blockId); @Override - public ByteBuf newInboundBuffer(ChannelHandlerContext ctx) { - // Use direct buffer if possible. - return ctx.alloc().ioBuffer(); - } - - @Override - public void inboundBufferUpdated(ChannelHandlerContext ctx, ByteBuf in) { + public void channelRead0(ChannelHandlerContext ctx, ByteBuf in) { // get header if (currentHeader == null && in.readableBytes() >= FileHeader.HEADER_SIZE()) { currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE())); diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServer.java b/core/src/main/java/org/apache/spark/network/netty/FileServer.java index a99af348ce..a7305cda88 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServer.java @@ -22,6 +22,9 @@ import java.net.InetSocketAddress; import io.netty.bootstrap.ServerBootstrap; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.nio.NioEventLoop; +import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.oio.OioServerSocketChannel; @@ -36,7 +39,8 @@ class FileServer { private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); - private ServerBootstrap bootstrap = null; + private EventLoopGroup bossGroup = null; + private EventLoopGroup workerGroup = null; private ChannelFuture channelFuture = null; private int port = 0; private Thread blockingThread = null; @@ -45,8 +49,11 @@ class FileServer { InetSocketAddress addr = new InetSocketAddress(port); // Configure the server. - bootstrap = new ServerBootstrap(); - bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup()) + bossGroup = new OioEventLoopGroup(); + workerGroup = new OioEventLoopGroup(); + + ServerBootstrap bootstrap = new ServerBootstrap(); + bootstrap.group(bossGroup, workerGroup) .channel(OioServerSocketChannel.class) .option(ChannelOption.SO_BACKLOG, 100) .option(ChannelOption.SO_RCVBUF, 1500) @@ -89,13 +96,19 @@ class FileServer { public void stop() { // Close the bound channel. if (channelFuture != null) { - channelFuture.channel().close(); + channelFuture.channel().close().awaitUninterruptibly(); channelFuture = null; } - // Shutdown bootstrap. - if (bootstrap != null) { - bootstrap.shutdown(); - bootstrap = null; + + // Shutdown event groups + if (bossGroup != null) { + bossGroup.shutdownGracefully(); + bossGroup = null; + } + + if (workerGroup != null) { + workerGroup.shutdownGracefully(); + workerGroup = null; } // TODO: Shutdown all accepted channels as well ? } diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java index 172c6e4b1c..f3009b4605 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java @@ -21,13 +21,13 @@ import java.io.File; import java.io.FileInputStream; import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelInboundMessageHandlerAdapter; +import io.netty.channel.SimpleChannelInboundHandler; import io.netty.channel.DefaultFileRegion; import org.apache.spark.storage.BlockId; import org.apache.spark.storage.FileSegment; -class FileServerHandler extends ChannelInboundMessageHandlerAdapter { +class FileServerHandler extends SimpleChannelInboundHandler { PathResolver pResolver; @@ -36,7 +36,7 @@ class FileServerHandler extends ChannelInboundMessageHandlerAdapter { } @Override - public void messageReceived(ChannelHandlerContext ctx, String blockIdString) { + public void channelRead0(ChannelHandlerContext ctx, String blockIdString) { BlockId blockId = BlockId.apply(blockIdString); FileSegment fileSegment = pResolver.getBlockLocation(blockId); // if getBlockLocation returns null, close the channel @@ -60,7 +60,7 @@ class FileServerHandler extends ChannelInboundMessageHandlerAdapter { int len = new Long(length).intValue(); ctx.write((new FileHeader(len, blockId)).buffer()); try { - ctx.sendFile(new DefaultFileRegion(new FileInputStream(file) + ctx.write(new DefaultFileRegion(new FileInputStream(file) .getChannel(), fileSegment.offset(), fileSegment.length())); } catch (Exception e) { e.printStackTrace(); diff --git a/pom.xml b/pom.xml index 57e843596f..0936ae53b4 100644 --- a/pom.xml +++ b/pom.xml @@ -282,7 +282,7 @@ io.netty netty-all - 4.0.0.CR1 + 4.0.13.Final org.apache.derby diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7bcbd90bd3..1df1abc9a3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -178,7 +178,7 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( - "io.netty" % "netty-all" % "4.0.0.CR1", + "io.netty" % "netty-all" % "4.0.13.Final", "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106", /** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */ "org.eclipse.jetty.orbit" % "javax.servlet" % "2.5.0.v201103041518" artifacts Artifact("javax.servlet", "jar", "jar"), -- cgit v1.2.3 From 9115a5de62dcb832569727773112a4688ef63f03 Mon Sep 17 00:00:00 2001 From: Binh Nguyen Date: Sat, 7 Dec 2013 09:15:31 -0800 Subject: Remove import * and fix some formatting --- core/src/main/java/org/apache/spark/network/netty/FileClient.java | 4 ++-- core/src/main/java/org/apache/spark/network/netty/FileServer.java | 7 ++----- 2 files changed, 4 insertions(+), 7 deletions(-) (limited to 'core/src') diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java index 95e25d94e6..6b7f6a9397 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClient.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClient.java @@ -20,10 +20,10 @@ package org.apache.spark.network.netty; import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoopGroup; import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.oio.OioSocketChannel; -import io.netty.util.concurrent.EventExecutorGroup; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,7 +83,7 @@ class FileClient { channel = null; } - if (group!=null) { + if (group != null) { group.shutdownGracefully(); group = null; bootstrap = null; diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServer.java b/core/src/main/java/org/apache/spark/network/netty/FileServer.java index a7305cda88..3fe7d69925 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServer.java @@ -17,20 +17,17 @@ package org.apache.spark.network.netty; -import java.net.InetSocketAddress; - import io.netty.bootstrap.ServerBootstrap; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelOption; import io.netty.channel.EventLoopGroup; -import io.netty.channel.nio.NioEventLoop; -import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.oio.OioServerSocketChannel; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.InetSocketAddress; + /** * Server that accept the path of a file an echo back its content. -- cgit v1.2.3 From 786f393a98f8771d0c20322cd50e553a895c7d60 Mon Sep 17 00:00:00 2001 From: Binh Nguyen Date: Sat, 7 Dec 2013 11:06:18 -0800 Subject: Fix imports order --- .../org/apache/spark/network/netty/FileClientChannelInitializer.java | 1 - core/src/main/java/org/apache/spark/network/netty/FileServer.java | 5 ++--- .../org/apache/spark/network/netty/FileServerChannelInitializer.java | 1 - 3 files changed, 2 insertions(+), 5 deletions(-) (limited to 'core/src') diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java index b16225026b..fb61be1c12 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java @@ -21,7 +21,6 @@ import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; import io.netty.handler.codec.string.StringEncoder; - class FileClientChannelInitializer extends ChannelInitializer { private final FileClientHandler fhandler; diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServer.java b/core/src/main/java/org/apache/spark/network/netty/FileServer.java index 3fe7d69925..aea7534459 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServer.java @@ -17,6 +17,8 @@ package org.apache.spark.network.netty; +import java.net.InetSocketAddress; + import io.netty.bootstrap.ServerBootstrap; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelOption; @@ -26,9 +28,6 @@ import io.netty.channel.socket.oio.OioServerSocketChannel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.net.InetSocketAddress; - - /** * Server that accept the path of a file an echo back its content. */ diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java index 833af1632d..f1f264c583 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java @@ -23,7 +23,6 @@ import io.netty.handler.codec.DelimiterBasedFrameDecoder; import io.netty.handler.codec.Delimiters; import io.netty.handler.codec.string.StringDecoder; - class FileServerChannelInitializer extends ChannelInitializer { PathResolver pResolver; -- cgit v1.2.3 From 94479673eb0ea839d5f6b6bd43c5abf75af7b9eb Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 26 Dec 2013 00:07:45 +0000 Subject: Fixed bug in PartitionAwareUnionRDD --- .../org/apache/spark/rdd/PartitionerAwareUnionRDD.scala | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala index 995042e590..3cbf3b4c4f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -9,8 +9,8 @@ class PartitionerAwareUnionRDDPartition( @transient val rdds: Seq[RDD[_]], val idx: Int ) extends Partition { - var parents = rdds.map(_.partitions(index)).toArray - + var parents = rdds.map(_.partitions(idx)).toArray + override val index = idx override def hashCode(): Int = idx @@ -42,7 +42,7 @@ class PartitionerAwareUnionRDD[T: ClassTag]( // Get the location where most of the partitions of parent RDDs are located override def getPreferredLocations(s: Partition): Seq[String] = { - logDebug("Getting preferred locations for " + this) + logDebug("Finding preferred location for " + this + ", partition " + s.index) val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].parents val locations = rdds.zip(parentPartitions).flatMap { case (rdd, part) => { @@ -51,11 +51,14 @@ class PartitionerAwareUnionRDD[T: ClassTag]( parentLocations } } - if (locations.isEmpty) { - Seq.empty + val location = if (locations.isEmpty) { + None } else { - Seq(locations.groupBy(x => x).map(x => (x._1, x._2.length)).maxBy(_._2)._1) + // Find the location where maximum number of parent partitions prefer + Some(locations.groupBy(x => x).maxBy(_._2.length)._1) } + logDebug("Selected location for " + this + ", partition " + s.index + " = " + location) + location.toSeq } override def compute(s: Partition, context: TaskContext): Iterator[T] = { -- cgit v1.2.3 From c529dceaffad0d0eb611ecb70fb5723dab43d0a1 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 25 Dec 2013 12:16:21 -0800 Subject: Avoid a lump of coal (NPE) in JobProgressListener's stocking. --- .../scala/org/apache/spark/ui/jobs/JobProgressListener.scala | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) (limited to 'core/src') 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 07a42f0503..2e51dd5a99 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 @@ -146,12 +146,9 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList // update duration y.taskTime += taskEnd.taskInfo.duration - taskEnd.taskMetrics.shuffleReadMetrics.foreach { shuffleRead => - y.shuffleRead += shuffleRead.remoteBytesRead - } - - taskEnd.taskMetrics.shuffleWriteMetrics.foreach { shuffleWrite => - y.shuffleWrite += shuffleWrite.shuffleBytesWritten + Option(taskEnd.taskMetrics).foreach { taskMetrics => + taskMetrics.shuffleReadMetrics.foreach { y.shuffleRead += _.remoteBytesRead } + taskMetrics.shuffleWriteMetrics.foreach { y.shuffleWrite += _.shuffleBytesWritten } } } case _ => {} -- cgit v1.2.3 From 5fde4566ea48e5c6d6c50af032a29eaded2d7c43 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 26 Dec 2013 14:33:37 -0800 Subject: Added Apache boilerplate and class docs to PartitionerAwareUnionRDD. --- .../spark/rdd/PartitionerAwareUnionRDD.scala | 36 ++++++++++++++++++++-- 1 file changed, 33 insertions(+), 3 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala index 3cbf3b4c4f..4c625d062e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -1,9 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.rdd -import org.apache.spark.{TaskContext, OneToOneDependency, SparkContext, Partition} import scala.reflect.ClassTag import java.io.{ObjectOutputStream, IOException} +import org.apache.spark.{TaskContext, OneToOneDependency, SparkContext, Partition} + +/** + * Class representing partitions of PartitionerAwareUnionRDD, which maintains the list of corresponding partitions + * of parent RDDs. + */ private[spark] class PartitionerAwareUnionRDDPartition( @transient val rdds: Seq[RDD[_]], @@ -22,6 +44,14 @@ class PartitionerAwareUnionRDDPartition( } } +/** + * Class representing an RDD that can take multiple RDDs partitioned by the same partitioner and + * unify them into a single RDD while preserving the partitioner. So m RDDs with p partitions each + * will be unified to a single RDD with p partitions and the same partitioner. The preferred + * location for each partition of the unified RDD will be the most common preferred location + * of the corresponding partitions of the parent RDDs. For example, location of partition 0 + * of the unified RDD will be where most of partition 0 of the parent RDDs are located. + */ private[spark] class PartitionerAwareUnionRDD[T: ClassTag]( sc: SparkContext, @@ -54,7 +84,7 @@ class PartitionerAwareUnionRDD[T: ClassTag]( val location = if (locations.isEmpty) { None } else { - // Find the location where maximum number of parent partitions prefer + // Find the location that maximum number of parent partitions prefer Some(locations.groupBy(x => x).maxBy(_._2.length)._1) } logDebug("Selected location for " + this + ", partition " + s.index + " = " + location) @@ -73,7 +103,7 @@ class PartitionerAwareUnionRDD[T: ClassTag]( rdds = null } - // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) + // Get the *current* preferred locations from the DAGScheduler (as opposed to the static ones) private def currPrefLocs(rdd: RDD[_], part: Partition): Seq[String] = { rdd.context.getPreferredLocs(rdd, part.index).map(tl => tl.host) } -- cgit v1.2.3 From 4f2fb761b0d0c756b777a516b2ffb480264ab943 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 26 Dec 2013 15:26:06 -0800 Subject: Decrease margin of left side of log page --- .../main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) (limited to 'core/src') 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 40d6bdb3fd..19aa800a95 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 @@ -140,12 +140,12 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I {linkToMaster}
    -
    {backButton}
    +
    {backButton}
    {range}
    -
    {nextButton}
    +
    {nextButton}

    -
    +
    {logText}
    -- cgit v1.2.3 From baaabcedc9225519c728ea34619b2c824aa9ac89 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Fri, 27 Dec 2013 11:12:36 -0800 Subject: Removed unused failed and causeOfFailure variables --- core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 6 ------ 1 file changed, 6 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index c676e73e03..26caeeda58 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -112,10 +112,6 @@ private[spark] class TaskSetManager( // Task index, start and finish time for each task attempt (indexed by task ID) val taskInfos = new HashMap[Long, TaskInfo] - // 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 @@ -556,8 +552,6 @@ private[spark] class TaskSetManager( } def abort(message: String) { - failed = true - causeOfFailure = message // TODO: Kill running tasks if we were not terminated due to a Mesos error sched.dagScheduler.taskSetFailed(taskSet, message) removeAllRunningTasks() -- cgit v1.2.3 From 2c5bade4ee6db747cbc7b0884094ad443834e3b1 Mon Sep 17 00:00:00 2001 From: Binh Nguyen Date: Fri, 27 Dec 2013 11:18:27 -0800 Subject: Fix failed unit tests Also clean up a bit. --- .../org/apache/spark/network/netty/FileClient.java | 25 ++++++++++++++-------- .../netty/FileServerChannelInitializer.java | 2 +- .../spark/network/netty/FileServerHandler.java | 10 ++++++--- 3 files changed, 24 insertions(+), 13 deletions(-) (limited to 'core/src') diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java index 6b7f6a9397..46d61503bc 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClient.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClient.java @@ -27,14 +27,17 @@ import io.netty.channel.socket.oio.OioSocketChannel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.concurrent.TimeUnit; + class FileClient { private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); - private FileClientHandler handler = null; + private final FileClientHandler handler; private Channel channel = null; private Bootstrap bootstrap = null; private EventLoopGroup group = null; - private int connectTimeout = 60*1000; // 1 min + private final int connectTimeout; + private final int sendTimeout = 60; // 1 min public FileClient(FileClientHandler handler, int connectTimeout) { this.handler = handler; @@ -43,7 +46,7 @@ class FileClient { public void init() { group = new OioEventLoopGroup(); - Bootstrap bootstrap = new Bootstrap(); + bootstrap = new Bootstrap(); bootstrap.group(group) .channel(OioSocketChannel.class) .option(ChannelOption.SO_KEEPALIVE, true) @@ -59,6 +62,7 @@ class FileClient { // ChannelFuture cf = channel.closeFuture(); //cf.addListener(new ChannelCloseListener(this)); } catch (InterruptedException e) { + LOG.warn("FileClient interrupted while trying to connect", e); close(); } } @@ -74,15 +78,18 @@ class FileClient { public void sendRequest(String file) { //assert(file == null); //assert(channel == null); - channel.write(file + "\r\n"); + try { + // Should be able to send the message to network link channel. + boolean bSent = channel.writeAndFlush(file + "\r\n").await(sendTimeout, TimeUnit.SECONDS); + if (!bSent) { + throw new RuntimeException("Failed to send"); + } + } catch (InterruptedException e) { + LOG.error("Error", e); + } } public void close() { - if(channel != null) { - channel.close().awaitUninterruptibly(); - channel = null; - } - if (group != null) { group.shutdownGracefully(); group = null; diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java index f1f264c583..3f15ff898f 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java @@ -35,7 +35,7 @@ class FileServerChannelInitializer extends ChannelInitializer { public void initChannel(SocketChannel channel) { channel.pipeline() .addLast("framer", new DelimiterBasedFrameDecoder(8192, Delimiters.lineDelimiter())) - .addLast("strDecoder", new StringDecoder()) + .addLast("stringDecoder", new StringDecoder()) .addLast("handler", new FileServerHandler(pResolver)); } } diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java index f3009b4605..e2d9391b4c 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java @@ -26,10 +26,14 @@ import io.netty.channel.DefaultFileRegion; import org.apache.spark.storage.BlockId; import org.apache.spark.storage.FileSegment; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class FileServerHandler extends SimpleChannelInboundHandler { - PathResolver pResolver; + private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); + + private final PathResolver pResolver; public FileServerHandler(PathResolver pResolver){ this.pResolver = pResolver; @@ -63,7 +67,7 @@ class FileServerHandler extends SimpleChannelInboundHandler { ctx.write(new DefaultFileRegion(new FileInputStream(file) .getChannel(), fileSegment.offset(), fileSegment.length())); } catch (Exception e) { - e.printStackTrace(); + LOG.error("Exception: ", e); } } else { ctx.write(new FileHeader(0, blockId).buffer()); @@ -73,7 +77,7 @@ class FileServerHandler extends SimpleChannelInboundHandler { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - cause.printStackTrace(); + LOG.error("Exception: ", cause); ctx.close(); } } -- cgit v1.2.3 From 8c81068e16d4485e7f35dfaf99de6ee99fd76678 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Fri, 27 Dec 2013 11:36:54 -0800 Subject: Fixed >100char lines in DAGScheduler.scala --- .../org/apache/spark/scheduler/DAGScheduler.scala | 42 ++++++++++++++-------- 1 file changed, 27 insertions(+), 15 deletions(-) (limited to 'core/src') 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 963d15b76d..2a131fde28 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -152,7 +152,8 @@ class DAGScheduler( val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done val running = new HashSet[Stage] // Stages we are running right now val failed = new HashSet[Stage] // Stages that must be resubmitted due to fetch failures - val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] // Missing tasks from each stage + // Missing tasks from each stage + val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] var lastFetchFailureTime: Long = 0 // Used to wait a bit to avoid repeated resubmits val activeJobs = new HashSet[ActiveJob] @@ -239,7 +240,8 @@ class DAGScheduler( shuffleToMapStage.get(shuffleDep.shuffleId) match { case Some(stage) => stage case None => - val stage = newOrUsedStage(shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId) + val stage = + newOrUsedStage(shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId) shuffleToMapStage(shuffleDep.shuffleId) = stage stage } @@ -248,7 +250,8 @@ class DAGScheduler( /** * Create a Stage -- either directly for use as a result stage, or as part of the (re)-creation * of a shuffle map stage in newOrUsedStage. The stage will be associated with the provided - * jobId. Production of shuffle map stages should always use newOrUsedStage, not newStage directly. + * jobId. Production of shuffle map stages should always use newOrUsedStage, not newStage + * directly. */ private def newStage( rdd: RDD[_], @@ -358,7 +361,8 @@ class DAGScheduler( stageIdToJobIds.getOrElseUpdate(s.id, new HashSet[Int]()) += jobId jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) += s.id val parents = getParentStages(s.rdd, jobId) - val parentsWithoutThisJobId = parents.filter(p => !stageIdToJobIds.get(p.id).exists(_.contains(jobId))) + val parentsWithoutThisJobId = parents.filter( + p => !stageIdToJobIds.get(p.id).exists(_.contains(jobId))) updateJobIdStageIdMapsList(parentsWithoutThisJobId ++ stages.tail) } } @@ -366,8 +370,9 @@ class DAGScheduler( } /** - * Removes job and any stages that are not needed by any other job. Returns the set of ids for stages that - * were removed. The associated tasks for those stages need to be cancelled if we got here via job cancellation. + * Removes job and any stages that are not needed by any other job. Returns the set of ids for + * stages that were removed. The associated tasks for those stages need to be cancelled if we + * got here via job cancellation. */ private def removeJobAndIndependentStages(jobId: Int): Set[Int] = { val registeredStages = jobIdToStageIds(jobId) @@ -378,7 +383,8 @@ class DAGScheduler( stageIdToJobIds.filterKeys(stageId => registeredStages.contains(stageId)).foreach { case (stageId, jobSet) => if (!jobSet.contains(jobId)) { - logError("Job %d not registered for stage %d even though that stage was registered for the job" + logError( + "Job %d not registered for stage %d even though that stage was registered for the job" .format(jobId, stageId)) } else { def removeStage(stageId: Int) { @@ -389,7 +395,8 @@ class DAGScheduler( running -= s } stageToInfos -= s - shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach(shuffleToMapStage.remove) + shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach( + shuffleToMapStage.remove) if (pendingTasks.contains(s) && !pendingTasks(s).isEmpty) { logDebug("Removing pending status for stage %d".format(stageId)) } @@ -407,7 +414,8 @@ class DAGScheduler( stageIdToStage -= stageId stageIdToJobIds -= stageId - logDebug("After removal of stage %d, remaining stages = %d".format(stageId, stageIdToStage.size)) + logDebug("After removal of stage %d, remaining stages = %d" + .format(stageId, stageIdToStage.size)) } jobSet -= jobId @@ -459,7 +467,8 @@ class DAGScheduler( assert(partitions.size > 0) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] val waiter = new JobWaiter(this, jobId, partitions.size, resultHandler) - eventProcessActor ! JobSubmitted(jobId, rdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties) + eventProcessActor ! JobSubmitted( + jobId, rdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties) waiter } @@ -494,7 +503,8 @@ class DAGScheduler( val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] val partitions = (0 until rdd.partitions.size).toArray val jobId = nextJobId.getAndIncrement() - eventProcessActor ! JobSubmitted(jobId, rdd, func2, partitions, allowLocal = false, callSite, listener, properties) + eventProcessActor ! JobSubmitted( + jobId, rdd, func2, partitions, allowLocal = false, callSite, listener, properties) listener.awaitResult() // Will throw an exception if the job fails } @@ -529,8 +539,8 @@ class DAGScheduler( case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => var finalStage: Stage = null try { - // New stage creation at times and if its not protected, the scheduler thread is killed. - // e.g. it can fail when jobs are run on HadoopRDD whose underlying hdfs files have been deleted + // New stage creation may throw an exception if, for example, jobs are run on a HadoopRDD + // whose underlying HDFS files have been deleted. finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite)) } catch { case e: Exception => @@ -563,7 +573,8 @@ class DAGScheduler( case JobGroupCancelled(groupId) => // Cancel all jobs belonging to this job group. // First finds all active jobs with this group id, and then kill stages for them. - val activeInGroup = activeJobs.filter(groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) + val activeInGroup = activeJobs.filter( + groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) val jobIds = activeInGroup.map(_.jobId) jobIds.foreach { handleJobCancellation } @@ -585,7 +596,8 @@ class DAGScheduler( stage <- stageIdToStage.get(task.stageId); stageInfo <- stageToInfos.get(stage) ) { - if (taskInfo.serializedSize > TASK_SIZE_TO_WARN * 1024 && !stageInfo.emittedTaskSizeWarning) { + if (taskInfo.serializedSize > TASK_SIZE_TO_WARN * 1024 && + !stageInfo.emittedTaskSizeWarning) { stageInfo.emittedTaskSizeWarning = true logWarning(("Stage %d (%s) contains a task of very large " + "size (%d KB). The maximum recommended task size is %d KB.").format( -- cgit v1.2.3 From 0c71ffe924a158608b1760477b883e4818d53af4 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Fri, 27 Dec 2013 12:19:38 -0800 Subject: Style fixes as per Reynold's review --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) (limited to 'core/src') 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 2a131fde28..c48a3d64ef 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -361,8 +361,8 @@ class DAGScheduler( stageIdToJobIds.getOrElseUpdate(s.id, new HashSet[Int]()) += jobId jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) += s.id val parents = getParentStages(s.rdd, jobId) - val parentsWithoutThisJobId = parents.filter( - p => !stageIdToJobIds.get(p.id).exists(_.contains(jobId))) + val parentsWithoutThisJobId = parents.filter(p => + !stageIdToJobIds.get(p.id).exists(_.contains(jobId))) updateJobIdStageIdMapsList(parentsWithoutThisJobId ++ stages.tail) } } @@ -395,8 +395,8 @@ class DAGScheduler( running -= s } stageToInfos -= s - shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach( - shuffleToMapStage.remove) + shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach(shuffleId => + shuffleToMapStage.remove(shuffleId)) if (pendingTasks.contains(s) && !pendingTasks(s).isEmpty) { logDebug("Removing pending status for stage %d".format(stageId)) } @@ -573,8 +573,8 @@ class DAGScheduler( case JobGroupCancelled(groupId) => // Cancel all jobs belonging to this job group. // First finds all active jobs with this group id, and then kill stages for them. - val activeInGroup = activeJobs.filter( - groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) + val activeInGroup = activeJobs.filter(activeJob => + groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) val jobIds = activeInGroup.map(_.jobId) jobIds.foreach { handleJobCancellation } -- cgit v1.2.3 From 8419148e5fa54b5e3dd6b95fd5176b71506a951e Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Fri, 27 Dec 2013 15:14:38 -0800 Subject: Remove unused hasPendingTasks methods --- core/src/main/scala/org/apache/spark/scheduler/Pool.scala | 4 ---- core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala | 1 - .../main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 7 ------- .../src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 4 ---- 4 files changed, 16 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 596f9adde9..1791242215 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -117,8 +117,4 @@ private[spark] class Pool( parent.decreaseRunningTasks(taskNum) } } - - override def hasPendingTasks(): Boolean = { - schedulableQueue.exists(_.hasPendingTasks()) - } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala index 1c7ea2dccc..d573e125a3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala @@ -42,5 +42,4 @@ private[spark] trait Schedulable { def executorLost(executorId: String, host: String): Unit def checkSpeculatableTasks(): Boolean def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] - def hasPendingTasks(): Boolean } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index dbac6b96ac..1b0f82fa24 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -365,13 +365,6 @@ private[spark] class TaskSchedulerImpl( } } - // Check for pending tasks in all our active jobs. - def hasPendingTasks: Boolean = { - synchronized { - rootPool.hasPendingTasks() - } - } - def executorLost(executorId: String, reason: ExecutorLossReason) { var failedExecutor: Option[String] = None diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index c676e73e03..9b95e418d8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -681,10 +681,6 @@ private[spark] class TaskSetManager( return foundTasks } - override def hasPendingTasks(): Boolean = { - numTasks > 0 && tasksSuccessful < numTasks - } - private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { val defaultWait = System.getProperty("spark.locality.wait", "3000") level match { -- cgit v1.2.3 From e17d7518ab10c218c9576db5754c7fa4cb92688a Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Fri, 27 Dec 2013 15:51:27 -0800 Subject: Removed unused OtherFailure TaskEndReason. --- core/src/main/scala/org/apache/spark/TaskEndReason.scala | 2 -- core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala | 4 ---- 2 files changed, 6 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index c1e5e04b31..faf6dcd618 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -53,5 +53,3 @@ private[spark] case class ExceptionFailure( private[spark] case object TaskResultLost extends TaskEndReason private[spark] case object TaskKilled extends TaskEndReason - -private[spark] case class OtherFailure(message: String) extends TaskEndReason 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 60927831a1..be5c95e59e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -328,10 +328,6 @@ class JobLogger(val user: String, val logDirName: String) task.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + mapId + " REDUCE_ID=" + reduceId stageLogInfo(task.stageId, taskStatus) - case OtherFailure(message) => - taskStatus += " STATUS=FAILURE TID=" + taskInfo.taskId + - " STAGE_ID=" + task.stageId + " INFO=" + message - stageLogInfo(task.stageId, taskStatus) case _ => } } -- cgit v1.2.3 From b4619e509bc3e06baa3b031ef2c1981d3bf02cbd Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Fri, 27 Dec 2013 17:44:32 -0800 Subject: Changed naming of StageCompleted event to be consistent The rest of the SparkListener events are named with "SparkListener" as the prefix of the name; this commit renames the StageCompleted event to SparkListenerStageCompleted for consistency. --- .../org/apache/spark/scheduler/DAGScheduler.scala | 2 +- .../org/apache/spark/scheduler/JobLogger.scala | 2 +- .../org/apache/spark/scheduler/SparkListener.scala | 22 ++++++++++++++-------- .../apache/spark/scheduler/SparkListenerBus.scala | 2 +- .../apache/spark/ui/jobs/JobProgressListener.scala | 2 +- .../apache/spark/scheduler/JobLoggerSuite.scala | 2 +- .../spark/scheduler/SparkListenerSuite.scala | 2 +- 7 files changed, 20 insertions(+), 14 deletions(-) (limited to 'core/src') 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 c48a3d64ef..7603eb292f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -827,7 +827,7 @@ class DAGScheduler( } logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) stageToInfos(stage).completionTime = Some(System.currentTimeMillis()) - listenerBus.post(StageCompleted(stageToInfos(stage))) + listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) running -= stage } event.reason match { 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 60927831a1..7858080e31 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -297,7 +297,7 @@ class JobLogger(val user: String, val logDirName: String) * When stage is completed, record stage completion status * @param stageCompleted Stage completed event */ - override def onStageCompleted(stageCompleted: StageCompleted) { + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { stageLogInfo(stageCompleted.stage.stageId, "STAGE_ID=%d STATUS=COMPLETED".format( stageCompleted.stage.stageId)) } 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 ee63b3c4a1..3becb4f068 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -27,7 +27,7 @@ sealed trait SparkListenerEvents case class SparkListenerStageSubmitted(stage: StageInfo, properties: Properties) extends SparkListenerEvents -case class StageCompleted(val stage: StageInfo) extends SparkListenerEvents +case class SparkListenerStageCompleted(val stage: StageInfo) extends SparkListenerEvents case class SparkListenerTaskStart(task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents @@ -47,7 +47,7 @@ trait SparkListener { /** * Called when a stage is completed, with information on the completed stage */ - def onStageCompleted(stageCompleted: StageCompleted) { } + def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { } /** * Called when a stage is submitted @@ -86,7 +86,7 @@ trait SparkListener { * Simple SparkListener that logs a few summary statistics when each stage completes */ class StatsReportListener extends SparkListener with Logging { - override def onStageCompleted(stageCompleted: StageCompleted) { + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { import org.apache.spark.scheduler.StatsReportListener._ implicit val sc = stageCompleted this.logInfo("Finished stage: " + stageCompleted.stage) @@ -119,13 +119,19 @@ object StatsReportListener extends Logging { val probabilities = percentiles.map{_ / 100.0} val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" - def extractDoubleDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Double]): Option[Distribution] = { + def extractDoubleDistribution( + stage:SparkListenerStageCompleted, + getMetric: (TaskInfo,TaskMetrics) => Option[Double]) + : Option[Distribution] = { Distribution(stage.stage.taskInfos.flatMap { case ((info,metric)) => getMetric(info, metric)}) } //is there some way to setup the types that I can get rid of this completely? - def extractLongDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Long]): Option[Distribution] = { + def extractLongDistribution( + stage:SparkListenerStageCompleted, + getMetric: (TaskInfo,TaskMetrics) => Option[Long]) + : Option[Distribution] = { extractDoubleDistribution(stage, (info, metric) => getMetric(info,metric).map{_.toDouble}) } @@ -147,12 +153,12 @@ object StatsReportListener extends Logging { } def showDistribution(heading:String, format: String, getMetric: (TaskInfo,TaskMetrics) => Option[Double]) - (implicit stage: StageCompleted) { + (implicit stage: SparkListenerStageCompleted) { showDistribution(heading, extractDoubleDistribution(stage, getMetric), format) } def showBytesDistribution(heading:String, getMetric: (TaskInfo,TaskMetrics) => Option[Long]) - (implicit stage: StageCompleted) { + (implicit stage: SparkListenerStageCompleted) { showBytesDistribution(heading, extractLongDistribution(stage, getMetric)) } @@ -169,7 +175,7 @@ object StatsReportListener extends Logging { } def showMillisDistribution(heading: String, getMetric: (TaskInfo, TaskMetrics) => Option[Long]) - (implicit stage: StageCompleted) { + (implicit stage: SparkListenerStageCompleted) { showMillisDistribution(heading, extractLongDistribution(stage, getMetric)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 85687ea330..e7defd768b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -41,7 +41,7 @@ private[spark] class SparkListenerBus() extends Logging { event match { case stageSubmitted: SparkListenerStageSubmitted => sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) - case stageCompleted: StageCompleted => + case stageCompleted: SparkListenerStageCompleted => sparkListeners.foreach(_.onStageCompleted(stageCompleted)) case jobStart: SparkListenerJobStart => sparkListeners.foreach(_.onJobStart(jobStart)) 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 2e51dd5a99..058bc2a2e5 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 @@ -61,7 +61,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList override def onJobStart(jobStart: SparkListenerJobStart) {} - override def onStageCompleted(stageCompleted: StageCompleted) = synchronized { + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { val stage = stageCompleted.stage poolToActiveStages(stageIdToPool(stage.stageId)) -= stage activeStages -= stage 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 002368ff55..d0bd20fc83 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -117,7 +117,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = onTaskEndCount += 1 override def onJobEnd(jobEnd: SparkListenerJobEnd) = onJobEndCount += 1 override def onJobStart(jobStart: SparkListenerJobStart) = onJobStartCount += 1 - override def onStageCompleted(stageCompleted: StageCompleted) = onStageCompletedCount += 1 + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = onStageCompletedCount += 1 override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = onStageSubmittedCount += 1 } sc.addSparkListener(joblogger) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index d4320e5e14..1a16e438c4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -174,7 +174,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc class SaveStageInfo extends SparkListener { val stageInfos = Buffer[StageInfo]() - override def onStageCompleted(stage: StageCompleted) { + override def onStageCompleted(stage: SparkListenerStageCompleted) { stageInfos += stage.stage } } -- cgit v1.2.3 From 642029e7f43322f84abe4f7f36bb0b1b95d8101d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 28 Dec 2013 17:13:15 -0500 Subject: Various fixes to configuration code - Got rid of global SparkContext.globalConf - Pass SparkConf to serializers and compression codecs - Made SparkConf public instead of private[spark] - Improved API of SparkContext and SparkConf - Switched executor environment vars to be passed through SparkConf - Fixed some places that were still using system properties - Fixed some tests, though others are still failing This still fails several tests in core, repl and streaming, likely due to properties not being set or cleared correctly (some of the tests run fine in isolation). --- .../main/scala/org/apache/spark/Accumulators.scala | 8 +- .../scala/org/apache/spark/MapOutputTracker.scala | 4 +- .../main/scala/org/apache/spark/Partitioner.scala | 6 +- .../main/scala/org/apache/spark/SparkConf.scala | 158 ++++++++++++++++----- .../main/scala/org/apache/spark/SparkContext.scala | 138 ++++++++++-------- .../src/main/scala/org/apache/spark/SparkEnv.scala | 11 +- .../apache/spark/api/java/JavaSparkContext.scala | 15 +- .../org/apache/spark/api/python/PythonRDD.scala | 6 +- .../org/apache/spark/broadcast/HttpBroadcast.scala | 33 +++-- .../apache/spark/broadcast/TorrentBroadcast.scala | 38 ++--- .../spark/deploy/ApplicationDescription.scala | 2 +- .../apache/spark/deploy/LocalSparkCluster.scala | 7 +- .../org/apache/spark/deploy/SparkHadoopUtil.scala | 14 +- .../apache/spark/deploy/client/TestClient.scala | 9 +- .../org/apache/spark/deploy/master/Master.scala | 36 ++--- .../deploy/master/SparkZooKeeperSession.scala | 2 +- .../master/ZooKeeperLeaderElectionAgent.scala | 2 +- .../deploy/master/ZooKeeperPersistenceEngine.scala | 2 +- .../org/apache/spark/deploy/worker/Worker.scala | 14 +- .../executor/CoarseGrainedExecutorBackend.scala | 4 +- .../scala/org/apache/spark/executor/Executor.scala | 17 +-- .../org/apache/spark/io/CompressionCodec.scala | 13 +- .../apache/spark/network/ConnectionManager.scala | 4 +- .../org/apache/spark/network/ReceiverTest.scala | 12 +- .../org/apache/spark/network/SenderTest.scala | 16 +-- .../apache/spark/network/netty/ShuffleCopier.scala | 6 +- .../scala/org/apache/spark/rdd/CheckpointRDD.scala | 7 +- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 2 +- .../scala/org/apache/spark/rdd/ShuffledRDD.scala | 2 +- .../scala/org/apache/spark/rdd/SubtractedRDD.scala | 2 +- .../org/apache/spark/scheduler/DAGScheduler.scala | 5 +- .../apache/spark/scheduler/InputFormatInfo.scala | 14 +- .../org/apache/spark/scheduler/JobLogger.scala | 2 +- .../org/apache/spark/scheduler/ResultTask.scala | 4 +- .../spark/scheduler/SchedulableBuilder.scala | 2 +- .../apache/spark/scheduler/ShuffleMapTask.scala | 6 +- .../spark/scheduler/cluster/ClusterScheduler.scala | 8 +- .../scheduler/cluster/ClusterTaskSetManager.scala | 12 +- .../cluster/CoarseGrainedSchedulerBackend.scala | 9 +- .../spark/scheduler/cluster/SchedulerBackend.scala | 3 - .../scheduler/cluster/SimrSchedulerBackend.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../spark/scheduler/cluster/TaskResultGetter.scala | 2 +- .../mesos/CoarseMesosSchedulerBackend.scala | 6 +- .../cluster/mesos/MesosSchedulerBackend.scala | 6 +- .../spark/scheduler/local/LocalScheduler.scala | 2 +- .../apache/spark/serializer/JavaSerializer.scala | 3 +- .../apache/spark/serializer/KryoSerializer.scala | 13 +- .../spark/serializer/SerializerManager.scala | 12 +- .../spark/storage/BlockFetcherIterator.scala | 2 +- .../org/apache/spark/storage/BlockManager.scala | 46 +++--- .../apache/spark/storage/BlockManagerMaster.scala | 4 +- .../spark/storage/BlockManagerMasterActor.scala | 4 +- .../apache/spark/storage/DiskBlockManager.scala | 2 +- .../apache/spark/storage/ShuffleBlockManager.scala | 9 +- .../apache/spark/storage/StoragePerfTester.scala | 2 +- .../org/apache/spark/storage/ThreadingTest.scala | 6 +- .../org/apache/spark/ui/UIWorkloadGenerator.scala | 17 ++- .../org/apache/spark/ui/env/EnvironmentUI.scala | 2 +- .../apache/spark/ui/jobs/JobProgressListener.scala | 4 +- .../scala/org/apache/spark/util/AkkaUtils.scala | 18 +-- .../org/apache/spark/util/MetadataCleaner.scala | 33 +++-- .../org/apache/spark/util/SizeEstimator.scala | 17 +-- .../main/scala/org/apache/spark/util/Utils.scala | 14 +- .../apache/spark/io/CompressionCodecSuite.scala | 8 +- .../cluster/ClusterTaskSetManagerSuite.scala | 2 +- .../spark/serializer/KryoSerializerSuite.scala | 14 +- .../apache/spark/storage/BlockManagerSuite.scala | 8 +- .../org/apache/spark/util/SizeEstimatorSuite.scala | 2 - .../spark/examples/bagel/WikipediaPageRank.scala | 4 +- .../bagel/WikipediaPageRankStandalone.scala | 4 +- .../apache/spark/mllib/recommendation/ALS.scala | 10 +- .../spark/deploy/yarn/ApplicationMaster.scala | 44 +++--- .../org/apache/spark/deploy/yarn/Client.scala | 38 ++--- .../apache/spark/deploy/yarn/ClientArguments.scala | 2 +- .../scala/org/apache/spark/repl/SparkILoop.scala | 16 ++- .../scala/org/apache/spark/repl/SparkIMain.scala | 4 +- .../org/apache/spark/streaming/Checkpoint.scala | 22 +-- .../scala/org/apache/spark/streaming/DStream.scala | 2 +- .../org/apache/spark/streaming/Scheduler.scala | 10 +- .../apache/spark/streaming/StreamingContext.scala | 25 ++-- .../streaming/dstream/NetworkInputDStream.scala | 6 +- .../spark/streaming/util/RawTextSender.scala | 4 +- .../apache/spark/streaming/InputStreamsSuite.scala | 6 +- .../org/apache/spark/streaming/TestSuiteBase.scala | 6 +- .../spark/deploy/yarn/ApplicationMaster.scala | 56 ++++---- .../org/apache/spark/deploy/yarn/Client.scala | 50 +++---- .../apache/spark/deploy/yarn/ClientArguments.scala | 2 +- 88 files changed, 692 insertions(+), 536 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 6e922a612a..5f73d234aa 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -41,7 +41,7 @@ class Accumulable[R, T] ( @transient initialValue: R, param: AccumulableParam[R, T]) extends Serializable { - + val id = Accumulators.newId @transient private var value_ = initialValue // Current value on master val zero = param.zero(initialValue) // Zero value to be passed to workers @@ -113,7 +113,7 @@ class Accumulable[R, T] ( def setValue(newValue: R) { this.value = newValue } - + // Called by Java when deserializing an object private def readObject(in: ObjectInputStream) { in.defaultReadObject() @@ -177,7 +177,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 JavaSerializer().newInstance() + val ser = new JavaSerializer(new SparkConf(false)).newInstance() val copy = ser.deserialize[R](ser.serialize(initialValue)) copy.clear() // In case it contained stuff copy @@ -215,7 +215,7 @@ private object Accumulators { val originals = Map[Long, Accumulable[_, _]]() val localAccums = Map[Thread, Map[Long, Accumulable[_, _]]]() var lastId: Long = 0 - + def newId: Long = synchronized { lastId += 1 return lastId diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 4520edb10d..cdae167aef 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -65,7 +65,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { protected val epochLock = new java.lang.Object private val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup) + new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf) // Send a message to the trackerActor and get its result within a default timeout, or // throw a SparkException if this fails. @@ -129,7 +129,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { if (fetchedStatuses == null) { // We won the race to fetch the output locs; do so logInfo("Doing the fetch; tracker actor = " + trackerActor) - val hostPort = Utils.localHostPort() + val hostPort = Utils.localHostPort(conf) // This try-finally prevents hangs due to timeouts: try { val fetchedBytes = diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 04c1eedfeb..7cb545a6be 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -32,8 +32,6 @@ abstract class Partitioner extends Serializable { } object Partitioner { - - import SparkContext.{globalConf => conf} /** * Choose a partitioner to use for a cogroup-like operation between a number of RDDs. * @@ -54,7 +52,7 @@ object Partitioner { for (r <- bySize if r.partitioner != None) { return r.partitioner.get } - if (conf.getOrElse("spark.default.parallelism", null) != null) { + if (rdd.context.conf.getOrElse("spark.default.parallelism", null) != null) { return new HashPartitioner(rdd.context.defaultParallelism) } else { return new HashPartitioner(bySize.head.partitions.size) @@ -92,7 +90,7 @@ class HashPartitioner(partitions: Int) extends Partitioner { class RangePartitioner[K <% Ordered[K]: ClassTag, V]( partitions: Int, @transient rdd: RDD[_ <: Product2[K,V]], - private val ascending: Boolean = true) + private val ascending: Boolean = true) extends Partitioner { // An array of upper bounds for the first (partitions - 1) partitions diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 9a4eefad2e..185ddb1fe5 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -1,71 +1,159 @@ package org.apache.spark -import scala.collection.JavaConversions._ -import scala.collection.concurrent.TrieMap +import scala.collection.JavaConverters._ +import scala.collection.mutable.HashMap import com.typesafe.config.ConfigFactory -private[spark] class SparkConf(loadClasspathRes: Boolean = true) extends Serializable { - @transient lazy val config = ConfigFactory.systemProperties() - .withFallback(ConfigFactory.parseResources("spark.conf")) - // TODO this should actually be synchronized - private val configMap = TrieMap[String, String]() +/** + * Configuration for a Spark application. Used to set various Spark parameters as key-value pairs. + * + * Most of the time, you would create a SparkConf object with `new SparkConf()`, which will load + * values from both the `spark.*` Java system properties and any `spark.conf` on your application's + * classpath (if it has one). In this case, system properties take priority over `spark.conf`, and + * any parameters you set directly on the `SparkConf` object take priority over both of those. + * + * For unit tests, you can also call `new SparkConf(false)` to skip loading external settings and + * get the same configuration no matter what is on the classpath. + * + * @param loadDefaults whether to load values from the system properties and classpath + */ +class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { - if (loadClasspathRes && !config.entrySet().isEmpty) { - for (e <- config.entrySet()) { - configMap += ((e.getKey, e.getValue.unwrapped().toString)) + /** Create a SparkConf that loads defaults from system properties and the classpath */ + def this() = this(true) + + private val settings = new HashMap[String, String]() + + if (loadDefaults) { + val typesafeConfig = ConfigFactory.systemProperties() + .withFallback(ConfigFactory.parseResources("spark.conf")) + for (e <- typesafeConfig.entrySet().asScala) { + settings(e.getKey) = e.getValue.unwrapped.toString } } - def setMasterUrl(master: String) = { - if (master != null) - configMap += (("spark.master", master)) + /** Set a configuration variable. */ + def set(key: String, value: String): SparkConf = { + settings(key) = value + this + } + + /** + * The master URL to connect to, such as "local" to run locally with one thread, "local[4]" to + * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster. + */ + def setMaster(master: String): SparkConf = { + if (master != null) { + settings("spark.master") = master + } this } - def setAppName(name: String) = { - if (name != null) - configMap += (("spark.appName", name)) + /** Set a name for your application. Shown in the Spark web UI. */ + def setAppName(name: String): SparkConf = { + if (name != null) { + settings("spark.appName") = name + } this } - def setJars(jars: Seq[String]) = { - if (!jars.isEmpty) - configMap += (("spark.jars", jars.mkString(","))) + /** Set JAR files to distribute to the cluster. */ + def setJars(jars: Seq[String]): SparkConf = { + if (!jars.isEmpty) { + settings("spark.jars") = jars.mkString(",") + } this } - def set(k: String, value: String) = { - configMap += ((k, value)) + /** Set JAR files to distribute to the cluster. (Java-friendly version.) */ + def setJars(jars: Array[String]): SparkConf = { + if (!jars.isEmpty) { + settings("spark.jars") = jars.mkString(",") + } this } - def setSparkHome(home: String) = { - if (home != null) - configMap += (("spark.home", home)) + /** Set an environment variable to be used when launching executors for this application. */ + def setExecutorEnv(variable: String, value: String): SparkConf = { + settings("spark.executorEnv." + variable) = value this } - def set(map: Seq[(String, String)]) = { - if (map != null && !map.isEmpty) - configMap ++= map + /** Set multiple environment variables to be used when launching executors. */ + def setExecutorEnv(variables: Seq[(String, String)]): SparkConf = { + for ((k, v) <- variables) { + setExecutorEnv(k, v) + } this } - def get(k: String): String = { - configMap(k) + /** + * Set multiple environment variables to be used when launching executors. + * (Java-friendly version.) + */ + def setExecutorEnv(variables: Array[(String, String)]): SparkConf = { + for ((k, v) <- variables) { + setExecutorEnv(k, v) + } + this } - def getAllConfiguration = configMap.clone.entrySet().iterator + /** + * Set the location where Spark is installed on worker nodes. This is only needed on Mesos if + * you are not using `spark.executor.uri` to disseminate the Spark binary distribution. + */ + def setSparkHome(home: String): SparkConf = { + if (home != null) { + settings("spark.home") = home + } + this + } + /** Set multiple parameters together */ + def setAll(settings: Traversable[(String, String)]) = { + this.settings ++= settings + this + } + + /** Set a parameter if it isn't already configured */ + def setIfMissing(key: String, value: String): SparkConf = { + if (!settings.contains(key)) { + settings(key) = value + } + this + } + + /** Get a parameter; throws an exception if it's not set */ + def get(key: String): String = { + settings(key) + } + + /** Get a parameter as an Option */ + def getOption(key: String): Option[String] = { + settings.get(key) + } + + /** Get all parameters as a list of pairs */ + def getAll: Seq[(String, String)] = settings.clone().toSeq + + /** Get a parameter, falling back to a default if not set */ def getOrElse(k: String, defaultValue: String): String = { - configMap.getOrElse(k, defaultValue) + settings.getOrElse(k, defaultValue) } - override def clone: SparkConf = { - val conf = new SparkConf(false) - conf.set(configMap.toSeq) - conf + /** Get all executor environment variables set on this SparkConf */ + def getExecutorEnv: Seq[(String, String)] = { + val prefix = "spark.executorEnv." + getAll.filter(pair => pair._1.startsWith(prefix)) + .map(pair => (pair._1.substring(prefix.length), pair._2)) } + /** Does the configuration contain a given parameter? */ + def contains(key: String): Boolean = settings.contains(key) + + /** Copy this object */ + override def clone: SparkConf = { + new SparkConf(false).setAll(settings) + } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 4300b07bdb..0567f7f437 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -22,8 +22,7 @@ import java.net.URI import java.util.Properties import java.util.concurrent.atomic.AtomicInteger -import scala.collection.{Map, immutable} -import scala.collection.JavaConversions._ +import scala.collection.{Map, Set, immutable} import scala.collection.generic.Growable import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -57,23 +56,32 @@ import org.apache.spark.util._ * 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. * - * @param conf a Spark Config object describing the context configuration. Any settings in this - * config overrides the default configs as well as system properties. - * - * @param environment Environment variables to set on worker nodes. + * @param conf_ a Spark Config object describing the application configuration. Any settings in + * this config overrides the default configs as well as system properties. + * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Can + * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] + * from a list of input files or InputFormats for the application. */ class SparkContext( - val conf: SparkConf, - val environment: Map[String, String] = Map(), + conf_ : SparkConf, // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, etc) - // too. This is typically generated from InputFormatInfo.computePreferredLocations .. host, set - // of data-local splits on host - val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = immutable.Map()) + // too. This is typically generated from InputFormatInfo.computePreferredLocations. It contains + // a map from hostname to a list of input format splits on the host. + val preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) extends Logging { /** - * 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. + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI + * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters + */ + def this(master: String, appName: String, conf: SparkConf) = + this(conf.setMaster(master).setAppName(appName)) + + /** + * Alternative constructor that allows setting common Spark properties directly * * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI. @@ -82,24 +90,42 @@ class SparkContext( * system or HDFS, HTTP, HTTPS, or FTP URLs. * @param environment Environment variables to set on worker nodes. */ - def this(master: String, appName: String, sparkHome: String = null, - jars: Seq[String] = Nil, environment: Map[String, String] = Map(), - preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = - immutable.Map()) = - this(new SparkConf(false).setAppName(appName).setMasterUrl(master) - .setJars(jars).set(environment.toSeq).setSparkHome(sparkHome), - environment, preferredNodeLocationData) + def this( + master: String, + appName: String, + sparkHome: String = null, + jars: Seq[String] = Nil, + environment: Map[String, String] = Map(), + preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) = + { + this( + new SparkConf() + .setMaster(master) + .setAppName(appName) + .setJars(jars) + .setExecutorEnv(environment.toSeq) + .setSparkHome(sparkHome), + preferredNodeLocationData) + } - // Set Spark driver host and port system properties - Try(conf.get("spark.driver.host")) - .getOrElse(conf.set("spark.driver.host", Utils.localHostName())) + val conf = conf_.clone() + + if (!conf.contains("spark.master")) { + throw new SparkException("A master URL must be set in your configuration") + } + if (!conf.contains("spark.appName")) { + throw new SparkException("An application must be set in your configuration") + } - Try(conf.get("spark.driver.port")) - .getOrElse(conf.set("spark.driver.port", "0")) + // Set Spark driver host and port system properties + conf.setIfMissing("spark.driver.host", Utils.localHostName()) + conf.setIfMissing("spark.driver.port", "0") - val jars: Seq[String] = if (conf.getOrElse("spark.jars", null) != null) { - conf.get("spark.jars").split(",") - } else null + val jars: Seq[String] = if (conf.contains("spark.jars")) { + conf.get("spark.jars").split(",").filter(_.size != 0) + } else { + null + } val master = conf.get("spark.master") val appName = conf.get("spark.appName") @@ -115,8 +141,8 @@ class SparkContext( conf.get("spark.driver.host"), conf.get("spark.driver.port").toInt, conf, - true, - isLocal) + isDriver = true, + isLocal = isLocal) SparkEnv.set(env) // Used to store a URL for each static file/jar together with the file's local timestamp @@ -125,7 +151,8 @@ class SparkContext( // Keeps track of all persisted RDDs private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]] - private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup) + private[spark] val metadataCleaner = + new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) // Initialize the Spark UI private[spark] val ui = new SparkUI(this) @@ -135,9 +162,14 @@ class SparkContext( // Add each JAR given through the constructor if (jars != null) { - jars.foreach { addJar(_) } + jars.foreach(addJar) } + private[spark] val executorMemory = conf.getOption("spark.executor.memory") + .orElse(Option(System.getenv("SPARK_MEM"))) + .map(Utils.memoryStringToMb) + .getOrElse(512) + // 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 @@ -148,10 +180,8 @@ class SparkContext( } } // Since memory can be set with a system property too, use that - executorEnvs("SPARK_MEM") = SparkContext.executorMemoryRequested + "m" - if (environment != null) { - executorEnvs ++= environment - } + executorEnvs("SPARK_MEM") = executorMemory + "m" + executorEnvs ++= conf.getExecutorEnv // Set SPARK_USER for user who is running SparkContext. val sparkUser = Option { @@ -183,12 +213,12 @@ class SparkContext( hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) } // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar" - Utils.getSystemProperties.foreach { case (key, value) => + conf.getAll.foreach { case (key, value) => if (key.startsWith("spark.hadoop.")) { hadoopConf.set(key.substring("spark.hadoop.".length), value) } } - val bufferSize = conf.getOrElse("spark.buffer.size", "65536") + val bufferSize = conf.getOrElse("spark.buffer.size", "65536") hadoopConf.set("io.file.buffer.size", bufferSize) hadoopConf } @@ -200,7 +230,7 @@ class SparkContext( override protected def childValue(parent: Properties): Properties = new Properties(parent) } - private[spark] def getLocalProperties(): Properties = localProperties.get() + private[spark] def getLocalProperties: Properties = localProperties.get() private[spark] def setLocalProperties(props: Properties) { localProperties.set(props) @@ -533,7 +563,7 @@ class SparkContext( // Fetch the file locally in case a job is executed locally. // Jobs that run through LocalScheduler will already fetch the required dependencies, // but jobs run in DAGScheduler.runLocally() will not so we must fetch the files here. - Utils.fetchFile(path, new File(SparkFiles.getRootDirectory)) + Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) } @@ -915,14 +945,6 @@ object SparkContext { private[spark] val SPARK_UNKNOWN_USER = "" - private lazy val conf = new SparkConf() - - private[spark] def globalConf = { - if (SparkEnv.get != null) { - SparkEnv.get.conf - } else conf - } - implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double) = 0.0 @@ -1031,18 +1053,10 @@ 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 - Try(globalConf.get("spark.executor.memory")).toOption - .orElse(Option(System.getenv("SPARK_MEM"))) - .map(Utils.memoryStringToMb) - .getOrElse(512) - } - // Creates a task scheduler based on a given master URL. Extracted for testing. - private - def createTaskScheduler(sc: SparkContext, master: String, appName: String): TaskScheduler = { + private def createTaskScheduler(sc: SparkContext, master: String, appName: String) + : TaskScheduler = + { // Regular expression used for local[N] master format val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks @@ -1076,10 +1090,10 @@ object SparkContext { case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang. val memoryPerSlaveInt = memoryPerSlave.toInt - if (SparkContext.executorMemoryRequested > memoryPerSlaveInt) { + if (sc.executorMemory > memoryPerSlaveInt) { throw new SparkException( "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format( - memoryPerSlaveInt, SparkContext.executorMemoryRequested)) + memoryPerSlaveInt, sc.executorMemory)) } val scheduler = new ClusterScheduler(sc) @@ -1137,7 +1151,7 @@ object SparkContext { case mesosUrl @ MESOS_REGEX(_) => MesosNativeLibrary.load() val scheduler = new ClusterScheduler(sc) - val coarseGrained = globalConf.getOrElse("spark.mesos.coarse", "false").toBoolean + val coarseGrained = sc.conf.getOrElse("spark.mesos.coarse", "false").toBoolean val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { new CoarseMesosSchedulerBackend(scheduler, sc, url, appName) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 78e4ae27b2..34fad3e763 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -107,7 +107,7 @@ object SparkEnv extends Logging { /** * Returns the ThreadLocal SparkEnv. */ - def getThreadLocal : SparkEnv = { + def getThreadLocal: SparkEnv = { env.get() } @@ -150,18 +150,19 @@ object SparkEnv extends Logging { val serializerManager = new SerializerManager val serializer = serializerManager.setDefault( - conf.getOrElse("spark.serializer", "org.apache.spark.serializer.JavaSerializer")) + conf.getOrElse("spark.serializer", "org.apache.spark.serializer.JavaSerializer"), conf) val closureSerializer = serializerManager.get( - conf.getOrElse("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")) + conf.getOrElse("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"), + conf) def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = { if (isDriver) { logInfo("Registering " + name) Left(actorSystem.actorOf(Props(newActor), name = name)) } else { - val driverHost: String = conf.getOrElse("spark.driver.host", "localhost") - val driverPort: Int = conf.getOrElse("spark.driver.port", "7077").toInt + val driverHost: String = conf.getOrElse("spark.driver.host", "localhost") + val driverPort: Int = conf.getOrElse("spark.driver.port", "7077").toInt Utils.checkHost(driverHost, "Expected hostname") val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name) logInfo("Connecting to " + name + ": " + url) 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 acf328aa6a..e03cf9d13a 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 @@ -29,17 +29,22 @@ 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, SparkContext} +import org.apache.spark._ import org.apache.spark.SparkContext.IntAccumulatorParam import org.apache.spark.SparkContext.DoubleAccumulatorParam import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import scala.Tuple2 /** * 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 { + /** + * @param conf a [[org.apache.spark.SparkConf]] object specifying Spark parameters + */ + def this(conf: SparkConf) = this(new SparkContext(conf)) /** * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). @@ -47,6 +52,14 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork */ def this(master: String, appName: String) = this(new SparkContext(master, appName)) + /** + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI + * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters + */ + def this(master: String, appName: String, conf: SparkConf) = + this(conf.setMaster(master).setAppName(appName)) + /** * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). * @param appName A name for your application, to display on the cluster web UI 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 d6eacfe23e..05fd824254 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 @@ -41,7 +41,7 @@ private[spark] class PythonRDD[T: ClassTag]( accumulator: Accumulator[JList[Array[Byte]]]) extends RDD[Array[Byte]](parent) { - val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt override def getPartitions = parent.partitions @@ -247,10 +247,10 @@ private class BytesToString extends org.apache.spark.api.java.function.Function[ */ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) extends AccumulatorParam[JList[Array[Byte]]] { - import SparkContext.{globalConf => conf} + Utils.checkHost(serverHost, "Expected hostname") - val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = SparkEnv.get.conf.getOrElse("spark.buffer.size", "65536").toInt override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList 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 cecb8c228b..47528bcee8 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedH private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { - + def value = value_ def blockId = BroadcastBlockId(id) @@ -40,7 +40,7 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false) } - if (!isLocal) { + if (!isLocal) { HttpBroadcast.write(id, value_) } @@ -81,41 +81,48 @@ private object HttpBroadcast extends Logging { private var serverUri: String = null private var server: HttpServer = null + // TODO: This shouldn't be a global variable so that multiple SparkContexts can coexist private val files = new TimeStampedHashSet[String] - private val cleaner = new MetadataCleaner(MetadataCleanerType.HTTP_BROADCAST, cleanup) + private var cleaner: MetadataCleaner = null - private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5,TimeUnit.MINUTES).toInt + private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES).toInt - private lazy val compressionCodec = CompressionCodec.createCodec() + private var compressionCodec: CompressionCodec = null def initialize(isDriver: Boolean, conf: SparkConf) { synchronized { if (!initialized) { - bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt - compress = conf.getOrElse("spark.broadcast.compress", "true").toBoolean + bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + compress = conf.getOrElse("spark.broadcast.compress", "true").toBoolean if (isDriver) { - createServer() + createServer(conf) conf.set("spark.httpBroadcast.uri", serverUri) } serverUri = conf.get("spark.httpBroadcast.uri") + cleaner = new MetadataCleaner(MetadataCleanerType.HTTP_BROADCAST, cleanup, conf) + compressionCodec = CompressionCodec.createCodec(conf) initialized = true } } } - + def stop() { synchronized { if (server != null) { server.stop() server = null } + if (cleaner != null) { + cleaner.cancel() + cleaner = null + } + compressionCodec = null initialized = false - cleaner.cancel() } } - private def createServer() { - broadcastDir = Utils.createTempDir(Utils.getLocalDir) + private def createServer(conf: SparkConf) { + broadcastDir = Utils.createTempDir(Utils.getLocalDir(conf)) server = new HttpServer(broadcastDir) server.start() serverUri = server.uri @@ -143,7 +150,7 @@ private object HttpBroadcast extends Logging { val in = { val httpConnection = new URL(url).openConnection() httpConnection.setReadTimeout(httpReadTimeout) - val inputStream = httpConnection.getInputStream() + val inputStream = httpConnection.getInputStream if (compress) { compressionCodec.compressedInputStream(inputStream) } else { diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 4a3801dc48..00ec3b971b 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -83,13 +83,13 @@ extends Broadcast[T](id) with Logging with Serializable { case None => val start = System.nanoTime logInfo("Started reading broadcast variable " + id) - + // Initialize @transient variables that will receive garbage values from the master. resetWorkerVariables() if (receiveBroadcast(id)) { value_ = TorrentBroadcast.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks) - + // Store the merged copy in cache so that the next worker doesn't need to rebuild it. // This creates a tradeoff between memory usage and latency. // Storing copy doubles the memory footprint; not storing doubles deserialization cost. @@ -122,14 +122,14 @@ extends Broadcast[T](id) with Logging with Serializable { while (attemptId > 0 && totalBlocks == -1) { TorrentBroadcast.synchronized { SparkEnv.get.blockManager.getSingle(metaId) match { - case Some(x) => + case Some(x) => val tInfo = x.asInstanceOf[TorrentInfo] totalBlocks = tInfo.totalBlocks totalBytes = tInfo.totalBytes arrayOfBlocks = new Array[TorrentBlock](totalBlocks) hasBlocks = 0 - - case None => + + case None => Thread.sleep(500) } } @@ -145,13 +145,13 @@ extends Broadcast[T](id) with Logging with Serializable { val pieceId = BroadcastHelperBlockId(broadcastId, "piece" + pid) TorrentBroadcast.synchronized { SparkEnv.get.blockManager.getSingle(pieceId) match { - case Some(x) => + case Some(x) => arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock] hasBlocks += 1 SparkEnv.get.blockManager.putSingle( pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, true) - - case None => + + case None => throw new SparkException("Failed to get " + pieceId + " of " + broadcastId) } } @@ -175,13 +175,13 @@ extends Logging { } } } - + def stop() { initialized = false } - lazy val BLOCK_SIZE = conf.getOrElse("spark.broadcast.blockSize", "4096").toInt * 1024 - + lazy val BLOCK_SIZE = conf.getOrElse("spark.broadcast.blockSize", "4096").toInt * 1024 + def blockifyObject[T](obj: T): TorrentInfo = { val byteArray = Utils.serialize[T](obj) val bais = new ByteArrayInputStream(byteArray) @@ -210,7 +210,7 @@ extends Logging { } def unBlockifyObject[T](arrayOfBlocks: Array[TorrentBlock], - totalBytes: Int, + totalBytes: Int, totalBlocks: Int): T = { var retByteArray = new Array[Byte](totalBytes) for (i <- 0 until totalBlocks) { @@ -223,22 +223,22 @@ extends Logging { } private[spark] case class TorrentBlock( - blockID: Int, - byteArray: Array[Byte]) + blockID: Int, + byteArray: Array[Byte]) extends Serializable private[spark] case class TorrentInfo( @transient arrayOfBlocks : Array[TorrentBlock], - totalBlocks: Int, - totalBytes: Int) + totalBlocks: Int, + totalBytes: Int) extends Serializable { - - @transient var hasBlocks = 0 + + @transient var hasBlocks = 0 } private[spark] class TorrentBroadcastFactory extends BroadcastFactory { - + def initialize(isDriver: Boolean, conf: SparkConf) { TorrentBroadcast.initialize(isDriver, conf) } def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index dda43dc018..19d393a0db 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -26,7 +26,7 @@ private[spark] class ApplicationDescription( val appUiUrl: String) extends Serializable { - val user = System.getProperty("user.name", "") + val user = System.getProperty("user.name", "") override def toString: String = "ApplicationDescription(" + name + ")" } 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 59d12a3e6f..ffc0cb0903 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -22,7 +22,7 @@ import akka.actor.ActorSystem import org.apache.spark.deploy.worker.Worker import org.apache.spark.deploy.master.Master import org.apache.spark.util.Utils -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import scala.collection.mutable.ArrayBuffer @@ -43,7 +43,8 @@ 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 conf = new SparkConf(false) + val (masterSystem, masterPort, _) = Master.startSystemAndActor(localHostname, 0, 0, conf) masterActorSystems += masterSystem val masterUrl = "spark://" + localHostname + ":" + masterPort val masters = Array(masterUrl) @@ -55,7 +56,7 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I workerActorSystems += workerSystem } - return masters + masters } def stop() { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 1c979ac3e0..4f402c1121 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -34,10 +34,10 @@ class SparkHadoopUtil { UserGroupInformation.setConfiguration(conf) def runAsUser(user: String)(func: () => Unit) { - // if we are already running as the user intended there is no reason to do the doAs. It + // if we are already running as the user intended there is no reason to do the doAs. It // will actually break secure HDFS access as it doesn't fill in the credentials. Also if - // the user is UNKNOWN then we shouldn't be creating a remote unknown user - // (this is actually the path spark on yarn takes) since SPARK_USER is initialized only + // the user is UNKNOWN then we shouldn't be creating a remote unknown user + // (this is actually the path spark on yarn takes) since SPARK_USER is initialized only // in SparkContext. val currentUser = Option(System.getProperty("user.name")). getOrElse(SparkContext.SPARK_UNKNOWN_USER) @@ -67,12 +67,14 @@ class SparkHadoopUtil { } object SparkHadoopUtil { - import SparkContext.{globalConf => conf} + private val hadoop = { - val yarnMode = java.lang.Boolean.valueOf(conf.getOrElse("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) + val yarnMode = java.lang.Boolean.valueOf(System.getenv("SPARK_YARN_MODE")) if (yarnMode) { try { - Class.forName("org.apache.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/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 426cf524ae..ef649fd80c 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 @@ -18,7 +18,7 @@ package org.apache.spark.deploy.client import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.{SparkConf, SparkContext, Logging} import org.apache.spark.deploy.{Command, ApplicationDescription} private[spark] object TestClient { @@ -46,11 +46,12 @@ private[spark] object TestClient { def main(args: Array[String]) { val url = args(0) val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, - conf = SparkContext.globalConf) + conf = new SparkConf) val desc = new ApplicationDescription( - "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored") + "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), + "dummy-spark-home", "ignored") val listener = new TestListener - val client = new Client(actorSystem, Array(url), desc, listener, SparkContext.globalConf) + val client = new Client(actorSystem, Array(url), desc, listener, new SparkConf) client.start() actorSystem.awaitTermination() } 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 2c162c4fa2..9c89e36b14 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 @@ -29,7 +29,7 @@ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension -import org.apache.spark.{SparkContext, Logging, SparkException} +import org.apache.spark.{SparkConf, SparkContext, Logging, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.MasterMessages._ @@ -38,14 +38,16 @@ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { - import context.dispatcher - val conf = SparkContext.globalConf + import context.dispatcher // to use Akka's scheduler.schedule() + + val conf = new SparkConf + val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - val WORKER_TIMEOUT = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 - val RETAINED_APPLICATIONS = conf.getOrElse("spark.deploy.retainedApplications", "200").toInt - val REAPER_ITERATIONS = conf.getOrElse("spark.dead.worker.persistence", "15").toInt - val RECOVERY_DIR = conf.getOrElse("spark.deploy.recoveryDirectory", "") - val RECOVERY_MODE = conf.getOrElse("spark.deploy.recoveryMode", "NONE") + val WORKER_TIMEOUT = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 + val RETAINED_APPLICATIONS = conf.getOrElse("spark.deploy.retainedApplications", "200").toInt + val REAPER_ITERATIONS = conf.getOrElse("spark.dead.worker.persistence", "15").toInt + val RECOVERY_DIR = conf.getOrElse("spark.deploy.recoveryDirectory", "") + val RECOVERY_MODE = conf.getOrElse("spark.deploy.recoveryMode", "NONE") var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] @@ -86,7 +88,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - val spreadOutApps = conf.getOrElse("spark.deploy.spreadOut", "true").toBoolean + val spreadOutApps = conf.getOrElse("spark.deploy.spreadOut", "true").toBoolean override def preStart() { logInfo("Starting Spark master at " + masterUrl) @@ -495,7 +497,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } else { if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT)) - workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it + workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it } } } @@ -507,8 +509,9 @@ private[spark] object Master { val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r def main(argStrings: Array[String]) { - val args = new MasterArguments(argStrings, SparkContext.globalConf) - val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort) + val conf = new SparkConf + val args = new MasterArguments(argStrings, conf) + val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort, conf) actorSystem.awaitTermination() } @@ -522,11 +525,12 @@ private[spark] object Master { } } - def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, - conf = SparkContext.globalConf) + def startSystemAndActor(host: String, port: Int, webUiPort: Int, conf: SparkConf) + : (ActorSystem, Int, Int) = + { + val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf) val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), actorName) - val timeout = AkkaUtils.askTimeout(SparkContext.globalConf) + val timeout = AkkaUtils.askTimeout(conf) val respFuture = actor.ask(RequestWebUIPort)(timeout) val resp = Await.result(respFuture, timeout).asInstanceOf[WebUIPortResponse] (actorSystem, boundPort, resp.webUIBoundPort) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala index 79d95b1a83..60c7a7c2d6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala @@ -37,7 +37,7 @@ import org.apache.spark.{SparkConf, Logging} */ private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher, conf: SparkConf) extends Logging { - val ZK_URL = conf.getOrElse("spark.deploy.zookeeper.url", "") + val ZK_URL = conf.getOrElse("spark.deploy.zookeeper.url", "") val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE val ZK_TIMEOUT_MILLIS = 30000 diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index df5bb368a2..a61597bbdf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -28,7 +28,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String, conf: SparkConf) extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging { - val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" + val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" private val watcher = new ZooKeeperWatcher() private val zk = new SparkZooKeeperSession(this, conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index c55b720422..245a558a59 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -27,7 +27,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) with SparkZooKeeperWatcher with Logging { - val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/master_status" val zk = new SparkZooKeeperSession(this, conf) 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 75a6e75c78..f844fcbbfc 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 @@ -55,7 +55,7 @@ private[spark] class Worker( val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs // Send a heartbeat every (heartbeat timeout) / 4 milliseconds - val HEARTBEAT_MILLIS = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 / 4 + val HEARTBEAT_MILLIS = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 / 4 val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 @@ -267,7 +267,7 @@ private[spark] class Worker( } private[spark] object Worker { - import org.apache.spark.SparkContext.globalConf + def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, @@ -276,14 +276,16 @@ private[spark] object Worker { } def startSystemAndActor(host: String, port: Int, webUiPort: Int, cores: Int, memory: Int, - masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None) - : (ActorSystem, Int) = { + masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None) + : (ActorSystem, Int) = + { // The LocalSparkCluster runs multiple local sparkWorkerX actor systems + val conf = new SparkConf val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("") val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, - conf = globalConf) + conf = conf) actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory, - masterUrls, workDir, globalConf), name = "Worker") + masterUrls, workDir, conf), name = "Worker") (actorSystem, boundPort) } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index c8319f6f6e..53a2b94a52 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import akka.actor._ import akka.remote._ -import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.{SparkConf, SparkContext, Logging} import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{Utils, AkkaUtils} @@ -98,7 +98,7 @@ private[spark] object CoarseGrainedExecutorBackend { // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor // before getting started with all our system properties, etc val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, - indestructible = true, conf = SparkContext.globalConf) + indestructible = true, conf = new SparkConf) // set it val sparkHostPort = hostname + ":" + boundPort // conf.set("spark.hostPort", sparkHostPort) 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 70fc30e993..a6eabc462b 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -57,17 +57,18 @@ private[spark] class Executor( // Make sure the local hostname we report matches the cluster scheduler's name for this host Utils.setCustomHostname(slaveHostname) + + // Set spark.* properties from executor arg val conf = new SparkConf(false) - // Set spark.* system properties from executor arg - for ((key, value) <- properties) { - conf.set(key, value) - } + conf.setAll(properties) // 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.getenv("SPARK_YARN_MODE"))) { - conf.set("spark.local.dir", getYarnLocalDirs()) + if (java.lang.Boolean.valueOf( + System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) + { + conf.set("spark.local.dir", getYarnLocalDirs()) } // Create our ClassLoader and set it on this thread @@ -331,12 +332,12 @@ private[spark] class Executor( // Fetch missing dependencies for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf) currentFiles(name) = timestamp } for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf) currentJars(name) = timestamp // Add it to our class loader val localName = name.split("/").last 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 8ef5019b6c..20402686a8 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -22,7 +22,7 @@ import java.io.{InputStream, OutputStream} import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} -import org.apache.spark.SparkConf +import org.apache.spark.{SparkEnv, SparkConf} /** @@ -38,16 +38,15 @@ trait CompressionCodec { private[spark] object CompressionCodec { - import org.apache.spark.SparkContext.globalConf - def createCodec(): CompressionCodec = { - createCodec(System.getProperty( + def createCodec(conf: SparkConf): CompressionCodec = { + createCodec(conf, conf.getOrElse( "spark.io.compression.codec", classOf[LZFCompressionCodec].getName)) } - def createCodec(codecName: String): CompressionCodec = { + def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { val ctor = Class.forName(codecName, true, Thread.currentThread.getContextClassLoader) .getConstructor(classOf[SparkConf]) - ctor.newInstance(globalConf).asInstanceOf[CompressionCodec] + ctor.newInstance(conf).asInstanceOf[CompressionCodec] } } @@ -72,7 +71,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = conf.getOrElse("spark.io.compression.snappy.block.size", "32768").toInt + val blockSize = conf.getOrElse("spark.io.compression.snappy.block.size", "32768").toInt new SnappyOutputStream(s, blockSize) } 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 3e902f8ac5..697096fa76 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -593,10 +593,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi private[spark] object ConnectionManager { - import SparkContext.globalConf - def main(args: Array[String]) { - val manager = new ConnectionManager(9999, globalConf) + val manager = new ConnectionManager(9999, new SparkConf) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { println("Received [" + msg + "] from [" + id + "]") None diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 4ca3cd390b..1c9d6030d6 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -19,19 +19,19 @@ package org.apache.spark.network import java.nio.ByteBuffer import java.net.InetAddress +import org.apache.spark.SparkConf private[spark] object ReceiverTest { - import org.apache.spark.SparkContext.globalConf def main(args: Array[String]) { - val manager = new ConnectionManager(9999, globalConf) + val manager = new ConnectionManager(9999, new SparkConf) println("Started connection manager with id = " + manager.id) - - manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + + manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { /*println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis)*/ - val buffer = ByteBuffer.wrap("response".getBytes()) + val buffer = ByteBuffer.wrap("response".getBytes) Some(Message.createBufferMessage(buffer, msg.id)) }) - Thread.currentThread.join() + Thread.currentThread.join() } } diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index 11c21fc1d5..dcbd183c88 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -19,29 +19,29 @@ package org.apache.spark.network import java.nio.ByteBuffer import java.net.InetAddress +import org.apache.spark.SparkConf private[spark] object SenderTest { - import org.apache.spark.SparkContext.globalConf def main(args: Array[String]) { - + if (args.length < 2) { println("Usage: SenderTest ") System.exit(1) } - + val targetHost = args(0) val targetPort = args(1).toInt val targetConnectionManagerId = new ConnectionManagerId(targetHost, targetPort) - val manager = new ConnectionManager(0, globalConf) + val manager = new ConnectionManager(0, new SparkConf) println("Started connection manager with id = " + manager.id) - manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { println("Received [" + msg + "] from [" + id + "]") None }) - - val size = 100 * 1024 * 1024 + + val size = 100 * 1024 * 1024 val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) buffer.flip @@ -50,7 +50,7 @@ private[spark] object SenderTest { val count = 100 (0 until count).foreach(i => { val dataMessage = Message.createBufferMessage(buffer.duplicate) - val startTime = System.currentTimeMillis + val startTime = System.currentTimeMillis /*println("Started timer at " + startTime)*/ val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) match { case Some(response) => diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala index 81b3104afd..db28ddf9ac 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala @@ -36,7 +36,7 @@ private[spark] class ShuffleCopier(conf: SparkConf) extends Logging { resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val connectTimeout = conf.getOrElse("spark.shuffle.netty.connect.timeout", "60000").toInt + val connectTimeout = conf.getOrElse("spark.shuffle.netty.connect.timeout", "60000").toInt val fc = new FileClient(handler, connectTimeout) try { @@ -104,10 +104,10 @@ private[spark] object ShuffleCopier extends Logging { val threads = if (args.length > 3) args(3).toInt else 10 val copiers = Executors.newFixedThreadPool(80) - val tasks = (for (i <- Range(0, threads)) yield { + val tasks = (for (i <- Range(0, threads)) yield { Executors.callable(new Runnable() { def run() { - val copier = new ShuffleCopier(SparkContext.globalConf) + val copier = new ShuffleCopier(new SparkConf) copier.getBlock(host, port, blockId, echoResultCollectCallBack) } }) diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 9fbe002748..2897c4b841 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -74,9 +74,6 @@ class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String) } private[spark] object CheckpointRDD extends Logging { - - import SparkContext.{globalConf => conf} - def splitIdToFile(splitId: Int): String = { "part-%05d".format(splitId) } @@ -94,7 +91,7 @@ private[spark] object CheckpointRDD extends Logging { throw new IOException("Checkpoint failed: temporary path " + tempOutputPath + " already exists") } - val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = env.conf.getOrElse("spark.buffer.size", "65536").toInt val fileOutputStream = if (blockSize < 0) { fs.create(tempOutputPath, false, bufferSize) @@ -124,7 +121,7 @@ private[spark] object CheckpointRDD extends Logging { def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = { val env = SparkEnv.get val fs = path.getFileSystem(SparkHadoopUtil.get.newConfiguration()) - val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = env.conf.getOrElse("spark.buffer.size", "65536").toInt val fileInputStream = fs.open(path, bufferSize) val serializer = env.serializer.newInstance() val deserializeStream = serializer.deserializeStream(fileInputStream) 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 911a002884..4ba4696fef 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -114,7 +114,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: map.changeValue(k, update) } - val ser = SparkEnv.get.serializerManager.get(serializerClass) + val ser = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf) for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { // Read them from the parent 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 3682c84598..0ccb309d0d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -59,7 +59,7 @@ class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[P] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context, - SparkEnv.get.serializerManager.get(serializerClass)) + SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf)) } override def clearDependencies() { 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 aab30b1bb4..4f90c7d3d6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -93,7 +93,7 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = { val partition = p.asInstanceOf[CoGroupPartition] - val serializer = SparkEnv.get.serializerManager.get(serializerClass) + val serializer = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf) val map = new JHashMap[K, ArrayBuffer[V]] def getSeq(k: K): ArrayBuffer[V] = { val seq = map.get(k) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 963d15b76d..77aa24e6b6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -158,7 +158,8 @@ class DAGScheduler( val activeJobs = new HashSet[ActiveJob] val resultStageToJob = new HashMap[Stage, ActiveJob] - val metadataCleaner = new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup) + val metadataCleaner = new MetadataCleaner( + MetadataCleanerType.DAG_SCHEDULER, this.cleanup, env.conf) /** * Starts the event processing actor. The actor has two responsibilities: @@ -529,7 +530,7 @@ class DAGScheduler( case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => var finalStage: Stage = null try { - // New stage creation at times and if its not protected, the scheduler thread is killed. + // New stage creation at times and if its not protected, the scheduler thread is killed. // e.g. it can fail when jobs are run on HadoopRDD whose underlying hdfs files have been deleted finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite)) } catch { diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index 1791ee660d..90eb8a747f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -32,7 +32,7 @@ import scala.collection.JavaConversions._ /** * Parses and holds information about inputFormat (and files) specified as a parameter. */ -class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], +class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], val path: String) extends Logging { var mapreduceInputFormat: Boolean = false @@ -40,7 +40,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl validate() - override def toString(): String = { + override def toString: String = { "InputFormatInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", path : " + path } @@ -125,7 +125,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl } private def findPreferredLocations(): Set[SplitInfo] = { - logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat + + logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat + ", inputFormatClazz : " + inputFormatClazz) if (mapreduceInputFormat) { return prefLocsFromMapreduceInputFormat() @@ -143,14 +143,14 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl object InputFormatInfo { /** Computes the preferred locations based on input(s) and returned a location to block map. - Typical use of this method for allocation would follow some algo like this - (which is what we currently do in YARN branch) : + Typical use of this method for allocation would follow some algo like this: + a) For each host, count number of splits hosted on that host. b) Decrement the currently allocated containers on that host. c) Compute rack info for each host and update rack -> count map based on (b). d) Allocate nodes based on (c) - e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node - (even if data locality on that is very high) : this is to prevent fragility of job if a single + e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node + (even if data locality on that is very high) : this is to prevent fragility of job if a single (or small set of) hosts go down. go to (a) until required nodes are allocated. 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 3f55cd5642..60927831a1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -41,7 +41,7 @@ import org.apache.spark.storage.StorageLevel class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { - def this() = this(System.getProperty("user.name", ""), + def this() = this(System.getProperty("user.name", ""), String.valueOf(System.currentTimeMillis())) private val logDir = 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 310ec62ca8..28f3ba53b8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -32,7 +32,9 @@ private[spark] object ResultTask { // expensive on the master node if it needs to launch thousands of tasks. val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]] - val metadataCleaner = new MetadataCleaner(MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues) + // TODO: This object shouldn't have global variables + val metadataCleaner = new MetadataCleaner( + MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues, new SparkConf) def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = { synchronized { diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index 9002d33cda..3cf995ea74 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -52,7 +52,7 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) extends SchedulableBuilder with Logging { - val schedulerAllocFile = Option(conf.get("spark.scheduler.allocation.file")) + val schedulerAllocFile = conf.getOption("spark.scheduler.allocation.file") val DEFAULT_SCHEDULER_FILE = "fairscheduler.xml" val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.pool" val DEFAULT_POOL_NAME = "default" 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 0f2deb4bcb..a37ead5632 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -37,7 +37,9 @@ private[spark] object ShuffleMapTask { // expensive on the master node if it needs to launch thousands of tasks. val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]] - val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues) + // TODO: This object shouldn't have global variables + val metadataCleaner = new MetadataCleaner( + MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues, new SparkConf) def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = { synchronized { @@ -152,7 +154,7 @@ private[spark] class ShuffleMapTask( try { // Obtain all the block writers for shuffle blocks. - val ser = SparkEnv.get.serializerManager.get(dep.serializerClass) + val ser = SparkEnv.get.serializerManager.get(dep.serializerClass, SparkEnv.get.conf) shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, partitionId, numOutputSplits, ser) // Write the map output to its associated buckets. diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala index 7e231ec44c..2707740d44 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala @@ -51,10 +51,10 @@ private[spark] class ClusterScheduler(val sc: SparkContext) { val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL = conf.getOrElse("spark.speculation.interval", "100").toLong + val SPECULATION_INTERVAL = conf.getOrElse("spark.speculation.interval", "100").toLong // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT = conf.getOrElse("spark.starvation.timeout", "15000").toLong + val STARVATION_TIMEOUT = conf.getOrElse("spark.starvation.timeout", "15000").toLong // ClusterTaskSetManagers are not thread safe, so any access to one should be synchronized // on this class. @@ -91,7 +91,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var rootPool: Pool = null // default scheduler is FIFO val schedulingMode: SchedulingMode = SchedulingMode.withName( - conf.getOrElse("spark.scheduler.mode", "FIFO")) + conf.getOrElse("spark.scheduler.mode", "FIFO")) // This is a var so that we can reset it for testing purposes. private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) @@ -120,7 +120,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) override def start() { backend.start() - if (conf.getOrElse("spark.speculation", "false").toBoolean) { + if (conf.getOrElse("spark.speculation", "false").toBoolean) { logInfo("Starting speculative execution thread") import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, 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 398b0cefbf..a46b16b92f 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 @@ -52,14 +52,14 @@ private[spark] class ClusterTaskSetManager( { val conf = sched.sc.conf // CPUs to request per task - val CPUS_PER_TASK = conf.getOrElse("spark.task.cpus", "1").toInt + val CPUS_PER_TASK = conf.getOrElse("spark.task.cpus", "1").toInt // Maximum times a task is allowed to fail before failing the job - val MAX_TASK_FAILURES = conf.getOrElse("spark.task.maxFailures", "4").toInt + val MAX_TASK_FAILURES = conf.getOrElse("spark.task.maxFailures", "4").toInt // Quantile of tasks at which to start speculation - val SPECULATION_QUANTILE = conf.getOrElse("spark.speculation.quantile", "0.75").toDouble - val SPECULATION_MULTIPLIER = conf.getOrElse("spark.speculation.multiplier", "1.5").toDouble + val SPECULATION_QUANTILE = conf.getOrElse("spark.speculation.quantile", "0.75").toDouble + val SPECULATION_MULTIPLIER = conf.getOrElse("spark.speculation.multiplier", "1.5").toDouble // Serializer for closures and tasks. val env = SparkEnv.get @@ -118,7 +118,7 @@ private[spark] class ClusterTaskSetManager( // How frequently to reprint duplicate exceptions in full, in milliseconds val EXCEPTION_PRINT_INTERVAL = - conf.getOrElse("spark.logging.exceptionPrintInterval", "10000").toLong + conf.getOrElse("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 @@ -678,7 +678,7 @@ private[spark] class ClusterTaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = conf.getOrElse("spark.locality.wait", "3000") + val defaultWait = conf.getOrElse("spark.locality.wait", "3000") level match { case TaskLocality.PROCESS_LOCAL => conf.getOrElse("spark.locality.wait.process", defaultWait).toLong diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 40555903ac..156b01b149 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -62,7 +62,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) // Periodically revive offers to allow delay scheduling to work - val reviveInterval = conf.getOrElse("spark.scheduler.revive.interval", "1000").toLong + val reviveInterval = conf.getOrElse("spark.scheduler.revive.interval", "1000").toLong import context.dispatcher context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } @@ -118,7 +118,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac removeExecutor(executorId, reason) sender ! true - case DisassociatedEvent(_, address, _) => + case DisassociatedEvent(_, address, _) => addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated")) } @@ -163,10 +163,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac override def start() { val properties = new ArrayBuffer[(String, String)] - val iterator = scheduler.sc.conf.getAllConfiguration - while (iterator.hasNext) { - val entry = iterator.next - val (key, value) = (entry.getKey.toString, entry.getValue.toString) + for ((key, value) <- scheduler.sc.conf.getAll) { if (key.startsWith("spark.") && !key.equals("spark.hostPort")) { properties += ((key, value)) } 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 5367218faa..65d3fc8187 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 @@ -31,7 +31,4 @@ private[spark] trait SchedulerBackend { def defaultParallelism(): Int def killTask(taskId: Long, executorId: String): Unit = throw new UnsupportedOperationException - - // Memory used by each executor (in megabytes) - protected val executorMemory: Int = SparkContext.executorMemoryRequested } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index d01329b2b3..d74f000ebb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -31,7 +31,7 @@ private[spark] class SimrSchedulerBackend( val tmpPath = new Path(driverFilePath + "_tmp") val filePath = new Path(driverFilePath) - val maxCores = conf.getOrElse("spark.simr.executor.cores", "1").toInt + val maxCores = conf.getOrElse("spark.simr.executor.cores", "1").toInt override def start() { super.start() 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 d6b8ac2d57..de69e3260d 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 @@ -49,7 +49,7 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse(null) - val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, + val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, sparkHome, "http://" + sc.ui.appUIAddress) client = new Client(sc.env.actorSystem, masters, appDesc, this, conf) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala index ff6cc37f1d..319c91b933 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler) extends Logging { - private val THREADS = sparkEnv.conf.getOrElse("spark.resultGetter.threads", "4").toInt + private val THREADS = sparkEnv.conf.getOrElse("spark.resultGetter.threads", "4").toInt private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( THREADS, "Result resolver thread") diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 2a3b0e15f7..1695374152 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -76,7 +76,7 @@ private[spark] class CoarseMesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) - val extraCoresPerSlave = conf.getOrElse("spark.mesos.extra.cores", "0").toInt + val extraCoresPerSlave = conf.getOrElse("spark.mesos.extra.cores", "0").toInt var nextMesosTaskId = 0 @@ -176,7 +176,7 @@ private[spark] class CoarseMesosSchedulerBackend( val slaveId = offer.getSlaveId.toString val mem = getResource(offer.getResourcesList, "mem") val cpus = getResource(offer.getResourcesList, "cpus").toInt - if (totalCoresAcquired < maxCores && mem >= executorMemory && cpus >= 1 && + if (totalCoresAcquired < maxCores && mem >= sc.executorMemory && cpus >= 1 && failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES && !slaveIdsWithExecutors.contains(slaveId)) { // Launch an executor on the slave @@ -192,7 +192,7 @@ private[spark] class CoarseMesosSchedulerBackend( .setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave)) .setName("Task " + taskId) .addResources(createResource("cpus", cpusToUse)) - .addResources(createResource("mem", executorMemory)) + .addResources(createResource("mem", sc.executorMemory)) .build() d.launchTasks(offer.getId, Collections.singletonList(task), filters) } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 9bb92b4f01..8dfd4d5fb3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -114,7 +114,7 @@ private[spark] class MesosSchedulerBackend( val memory = Resource.newBuilder() .setName("mem") .setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(executorMemory).build()) + .setScalar(Value.Scalar.newBuilder().setValue(sc.executorMemory).build()) .build() ExecutorInfo.newBuilder() .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) @@ -199,7 +199,7 @@ private[spark] class MesosSchedulerBackend( def enoughMemory(o: Offer) = { val mem = getResource(o.getResourcesList, "mem") val slaveId = o.getSlaveId.getValue - mem >= executorMemory || slaveIdsWithExecutors.contains(slaveId) + mem >= sc.executorMemory || slaveIdsWithExecutors.contains(slaveId) } for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) { @@ -341,5 +341,5 @@ private[spark] class MesosSchedulerBackend( } // TODO: query Mesos for number of cores - override def defaultParallelism() = sc.conf.getOrElse("spark.default.parallelism", "8").toInt + override def defaultParallelism() = sc.conf.getOrElse("spark.default.parallelism", "8").toInt } 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 6069c1db3a..8498cffd31 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 @@ -92,7 +92,7 @@ private[spark] class LocalScheduler(val threads: Int, val maxFailures: Int, val var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null val schedulingMode: SchedulingMode = SchedulingMode.withName( - conf.getOrElse("spark.scheduler.mode", "FIFO")) + conf.getOrElse("spark.scheduler.mode", "FIFO")) val activeTaskSets = new HashMap[String, LocalTaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 4de81617b1..5d3d43623d 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -21,6 +21,7 @@ import java.io._ import java.nio.ByteBuffer import org.apache.spark.util.ByteBufferInputStream +import org.apache.spark.SparkConf private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream { val objOut = new ObjectOutputStream(out) @@ -77,6 +78,6 @@ private[spark] class JavaSerializerInstance extends SerializerInstance { /** * A Spark serializer that uses Java's built-in serialization. */ -class JavaSerializer extends Serializer { +class JavaSerializer(conf: SparkConf) extends Serializer { def newInstance(): SerializerInstance = new JavaSerializerInstance } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 17cec81038..2367f3f521 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -25,20 +25,21 @@ import com.esotericsoftware.kryo.{KryoException, Kryo} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.twitter.chill.{EmptyScalaKryoInstantiator, AllScalaRegistrar} -import org.apache.spark.{SparkContext, SparkConf, SerializableWritable, Logging} +import org.apache.spark._ import org.apache.spark.broadcast.HttpBroadcast import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ import scala.util.Try +import org.apache.spark.storage.PutBlock +import org.apache.spark.storage.GetBlock +import org.apache.spark.storage.GotBlock /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. */ -class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging { - - private val conf = SparkContext.globalConf +class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer with Logging { private val bufferSize = { - conf.getOrElse("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 + conf.getOrElse("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 } def newKryoOutput() = new KryoOutput(bufferSize) @@ -50,7 +51,7 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. // Do this before we invoke the user registrator so the user registrator can override this. - kryo.setReferences(conf.getOrElse("spark.kryo.referenceTracking", "true").toBoolean) + kryo.setReferences(conf.getOrElse("spark.kryo.referenceTracking", "true").toBoolean) for (cls <- KryoSerializer.toRegister) kryo.register(cls) diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index 2955986fec..22465272f3 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.serializer import java.util.concurrent.ConcurrentHashMap +import org.apache.spark.SparkConf /** @@ -32,12 +33,12 @@ private[spark] class SerializerManager { def default = _default - def setDefault(clsName: String): Serializer = { - _default = get(clsName) + def setDefault(clsName: String, conf: SparkConf): Serializer = { + _default = get(clsName, conf) _default } - def get(clsName: String): Serializer = { + def get(clsName: String, conf: SparkConf): Serializer = { if (clsName == null) { default } else { @@ -51,8 +52,9 @@ private[spark] class SerializerManager { serializer = serializers.get(clsName) if (serializer == null) { val clsLoader = Thread.currentThread.getContextClassLoader - serializer = - Class.forName(clsName, true, clsLoader).newInstance().asInstanceOf[Serializer] + val cls = Class.forName(clsName, true, clsLoader) + val constructor = cls.getConstructor(classOf[SparkConf]) + serializer = constructor.newInstance(conf).asInstanceOf[Serializer] serializers.put(clsName, serializer) } serializer 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 ee2ae471a9..3b25f68ca8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -327,7 +327,7 @@ object BlockFetcherIterator { fetchRequestsSync.put(request) } - copiers = startCopiers(conf.getOrElse("spark.shuffle.copier.threads", "6").toInt) + copiers = startCopiers(conf.getOrElse("spark.shuffle.copier.threads", "6").toInt) logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) 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 ffd166e93a..16ee208617 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -58,8 +58,8 @@ private[spark] class BlockManager( // If we use Netty for shuffle, start a new Netty-based shuffle sender service. private val nettyPort: Int = { - val useNetty = conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean - val nettyPortConfig = conf.getOrElse("spark.shuffle.sender.port", "0").toInt + val useNetty = conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean + val nettyPortConfig = conf.getOrElse("spark.shuffle.sender.port", "0").toInt if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 } @@ -72,18 +72,18 @@ private[spark] class BlockManager( // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory // for receiving shuffle outputs) val maxBytesInFlight = - conf.getOrElse("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 + conf.getOrElse("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 // Whether to compress broadcast variables that are stored - val compressBroadcast = conf.getOrElse("spark.broadcast.compress", "true").toBoolean + val compressBroadcast = conf.getOrElse("spark.broadcast.compress", "true").toBoolean // Whether to compress shuffle output that are stored - val compressShuffle = conf.getOrElse("spark.shuffle.compress", "true").toBoolean + val compressShuffle = conf.getOrElse("spark.shuffle.compress", "true").toBoolean // Whether to compress RDD partitions that are stored serialized - val compressRdds = conf.getOrElse("spark.rdd.compress", "false").toBoolean + val compressRdds = conf.getOrElse("spark.rdd.compress", "false").toBoolean - val heartBeatFrequency = BlockManager.getHeartBeatFrequencyFromSystemProperties + val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf) - val hostPort = Utils.localHostPort() + val hostPort = Utils.localHostPort(conf) val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) @@ -101,8 +101,11 @@ private[spark] class BlockManager( var heartBeatTask: Cancellable = null - private val metadataCleaner = new MetadataCleaner(MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks) - private val broadcastCleaner = new MetadataCleaner(MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks) + private val metadataCleaner = new MetadataCleaner( + MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks, conf) + private val broadcastCleaner = new MetadataCleaner( + MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks, conf) + initialize() // The compression codec to use. Note that the "lazy" val is necessary because we want to delay @@ -110,14 +113,14 @@ private[spark] class BlockManager( // program could be using a user-defined codec in a third party jar, which is loaded in // Executor.updateDependencies. When the BlockManager is initialized, user level jars hasn't been // loaded yet. - private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec() + private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec(conf) /** * Construct a BlockManager with a memory limit set based on system properties. */ def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, serializer: Serializer, conf: SparkConf) = { - this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties, conf) + this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), conf) } /** @@ -127,7 +130,7 @@ private[spark] class BlockManager( private def initialize() { master.registerBlockManager(blockManagerId, maxMemory, slaveActor) BlockManagerWorker.startBlockManagerWorker(this) - if (!BlockManager.getDisableHeartBeatsForTesting) { + if (!BlockManager.getDisableHeartBeatsForTesting(conf)) { heartBeatTask = actorSystem.scheduler.schedule(0.seconds, heartBeatFrequency.milliseconds) { heartBeat() } @@ -440,7 +443,7 @@ private[spark] class BlockManager( : BlockFetcherIterator = { val iter = - if (conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean) { + if (conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean) { new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) } else { new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer) @@ -466,7 +469,7 @@ private[spark] class BlockManager( def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int) : BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) - val syncWrites = conf.getOrElse("spark.shuffle.sync", "false").toBoolean + val syncWrites = conf.getOrElse("spark.shuffle.sync", "false").toBoolean new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites) } @@ -858,19 +861,18 @@ private[spark] class BlockManager( private[spark] object BlockManager extends Logging { - import org.apache.spark.SparkContext.{globalConf => conf} val ID_GENERATOR = new IdGenerator - def getMaxMemoryFromSystemProperties: Long = { - val memoryFraction = conf.getOrElse("spark.storage.memoryFraction", "0.66").toDouble + def getMaxMemory(conf: SparkConf): Long = { + val memoryFraction = conf.getOrElse("spark.storage.memoryFraction", "0.66").toDouble (Runtime.getRuntime.maxMemory * memoryFraction).toLong } - def getHeartBeatFrequencyFromSystemProperties: Long = - conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 + def getHeartBeatFrequency(conf: SparkConf): Long = + conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 - def getDisableHeartBeatsForTesting: Boolean = - conf.getOrElse("spark.test.disableBlockManagerHeartBeat", "false").toBoolean + def getDisableHeartBeatsForTesting(conf: SparkConf): Boolean = + conf.getOrElse("spark.test.disableBlockManagerHeartBeat", "false").toBoolean /** * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index fde7d63a68..8e4a88b20a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -31,8 +31,8 @@ private[spark] class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection], conf: SparkConf) extends Logging { - val AKKA_RETRY_ATTEMPTS: Int = conf.getOrElse("spark.akka.num.retries", "3").toInt - val AKKA_RETRY_INTERVAL_MS: Int = conf.getOrElse("spark.akka.retry.wait", "3000").toInt + val AKKA_RETRY_ATTEMPTS: Int = conf.getOrElse("spark.akka.num.retries", "3").toInt + val AKKA_RETRY_INTERVAL_MS: Int = conf.getOrElse("spark.akka.retry.wait", "3000").toInt val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" 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 05502e4451..73a1da2de6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -53,7 +53,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act initLogging() val slaveTimeout = conf.getOrElse("spark.storage.blockManagerSlaveTimeoutMs", - "" + (BlockManager.getHeartBeatFrequencyFromSystemProperties * 3)).toLong + "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong val checkTimeoutInterval = conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong @@ -61,7 +61,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act var timeoutCheckingTask: Cancellable = null override def preStart() { - if (!BlockManager.getDisableHeartBeatsForTesting) { + if (!BlockManager.getDisableHeartBeatsForTesting(conf)) { import context.dispatcher timeoutCheckingTask = context.system.scheduler.schedule( 0.seconds, checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 8f528babd4..7697092e1b 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -38,7 +38,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD extends PathResolver with Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 - private val subDirsPerLocalDir = shuffleManager.conf.getOrElse("spark.diskStore.subDirectories", "64").toInt + private val subDirsPerLocalDir = shuffleManager.conf.getOrElse("spark.diskStore.subDirectories", "64").toInt // Create one local directory for each path mentioned in spark.local.dir; then, inside this // directory, create multiple subdirectories that we will hash files into, in order to avoid diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 850d3178dd..f592df283a 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -62,12 +62,13 @@ private[spark] trait ShuffleWriterGroup { private[spark] class ShuffleBlockManager(blockManager: BlockManager) { def conf = blockManager.conf + // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. // TODO: Remove this once the shuffle file consolidation feature is stable. val consolidateShuffleFiles = - conf.getOrElse("spark.shuffle.consolidateFiles", "false").toBoolean + conf.getOrElse("spark.shuffle.consolidateFiles", "false").toBoolean - private val bufferSize = conf.getOrElse("spark.shuffle.file.buffer.kb", "100").toInt * 1024 + private val bufferSize = conf.getOrElse("spark.shuffle.file.buffer.kb", "100").toInt * 1024 /** * Contains all the state related to a particular shuffle. This includes a pool of unused @@ -82,8 +83,8 @@ class ShuffleBlockManager(blockManager: BlockManager) { type ShuffleId = Int private val shuffleStates = new TimeStampedHashMap[ShuffleId, ShuffleState] - private - val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup) + private val metadataCleaner = + new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup, conf) def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer) = { new ShuffleWriterGroup { diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala index d52b3d8284..40734aab49 100644 --- a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala +++ b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala @@ -56,7 +56,7 @@ object StoragePerfTester { def writeOutputBytes(mapId: Int, total: AtomicLong) = { val shuffle = blockManager.shuffleBlockManager.forMapTask(1, mapId, numOutputSplits, - new KryoSerializer()) + new KryoSerializer(sc.conf)) val writers = shuffle.writers for (i <- 1 to recordsPerMap) { writers(i % numOutputSplits).write(writeData) 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 b3b3893393..dca98c6c05 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -22,7 +22,7 @@ import akka.actor._ import java.util.concurrent.ArrayBlockingQueue import util.Random import org.apache.spark.serializer.KryoSerializer -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} /** * This class tests the BlockManager and MemoryStore for thread safety and @@ -92,8 +92,8 @@ private[spark] object ThreadingTest { def main(args: Array[String]) { System.setProperty("spark.kryoserializer.buffer.mb", "1") val actorSystem = ActorSystem("test") - val conf = SparkContext.globalConf - val serializer = new KryoSerializer + val conf = new SparkConf() + val serializer = new KryoSerializer(conf) val blockManagerMaster = new BlockManagerMaster( Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf) val blockManager = new BlockManager( diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index 14751e8e8e..58d47a201d 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -19,7 +19,7 @@ package org.apache.spark.ui import scala.util.Random -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.scheduler.SchedulingMode @@ -31,7 +31,6 @@ import org.apache.spark.scheduler.SchedulingMode */ private[spark] object UIWorkloadGenerator { - import SparkContext.{globalConf => conf} val NUM_PARTITIONS = 100 val INTER_JOB_WAIT_MS = 5000 @@ -40,14 +39,14 @@ private[spark] object UIWorkloadGenerator { println("usage: ./spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") System.exit(1) } - val master = args(0) - val schedulingMode = SchedulingMode.withName(args(1)) - val appName = "Spark UI Tester" + val conf = new SparkConf().setMaster(args(0)).setAppName("Spark UI tester") + + val schedulingMode = SchedulingMode.withName(args(1)) if (schedulingMode == SchedulingMode.FAIR) { - conf.set("spark.scheduler.mode", "FAIR") + conf.set("spark.scheduler.mode", "FAIR") } - val sc = new SparkContext(master, appName) + val sc = new SparkContext(conf) def setProperties(s: String) = { if(schedulingMode == SchedulingMode.FAIR) { @@ -57,11 +56,11 @@ private[spark] object UIWorkloadGenerator { } val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS) - def nextFloat() = (new Random()).nextFloat() + def nextFloat() = new Random().nextFloat() val jobs = Seq[(String, () => Long)]( ("Count", baseData.count), - ("Cache and Count", baseData.map(x => x).cache.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", { diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index b637d37517..91fa00a66c 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -63,7 +63,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) { UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties, fixedWidth = true) val classPathEntries = classPathProperty._2 - .split(sc.conf.getOrElse("path.separator", ":")) + .split(sc.conf.getOrElse("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")} 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 f01a1380b9..6ff8e9fb14 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 @@ -33,7 +33,7 @@ import org.apache.spark.scheduler._ */ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { // How many stages to remember - val RETAINED_STAGES = sc.conf.getOrElse("spark.ui.retained_stages", "1000").toInt + val RETAINED_STAGES = sc.conf.getOrElse("spark.ui.retained_stages", "1000").toInt val DEFAULT_POOL_NAME = "default" val stageIdToPool = new HashMap[Int, String]() @@ -105,7 +105,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[StageInfo]()) stages += stage } - + override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { val sid = taskStart.task.stageId val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 76febd5702..58b26f7f12 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -41,19 +41,19 @@ private[spark] object AkkaUtils { def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false, conf: SparkConf): (ActorSystem, Int) = { - val akkaThreads = conf.getOrElse("spark.akka.threads", "4").toInt - val akkaBatchSize = conf.getOrElse("spark.akka.batchSize", "15").toInt + val akkaThreads = conf.getOrElse("spark.akka.threads", "4").toInt + val akkaBatchSize = conf.getOrElse("spark.akka.batchSize", "15").toInt - val akkaTimeout = conf.getOrElse("spark.akka.timeout", "100").toInt + val akkaTimeout = conf.getOrElse("spark.akka.timeout", "100").toInt - val akkaFrameSize = conf.getOrElse("spark.akka.frameSize", "10").toInt + val akkaFrameSize = conf.getOrElse("spark.akka.frameSize", "10").toInt val lifecycleEvents = - if (conf.getOrElse("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" + if (conf.getOrElse("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - val akkaHeartBeatPauses = conf.getOrElse("spark.akka.heartbeat.pauses", "600").toInt + val akkaHeartBeatPauses = conf.getOrElse("spark.akka.heartbeat.pauses", "600").toInt val akkaFailureDetector = - conf.getOrElse("spark.akka.failure-detector.threshold", "300.0").toDouble - val akkaHeartBeatInterval = conf.getOrElse("spark.akka.heartbeat.interval", "1000").toInt + conf.getOrElse("spark.akka.failure-detector.threshold", "300.0").toDouble + val akkaHeartBeatInterval = conf.getOrElse("spark.akka.heartbeat.interval", "1000").toInt val akkaConf = ConfigFactory.parseString( s""" @@ -89,6 +89,6 @@ private[spark] object AkkaUtils { /** Returns the default Spark timeout to use for Akka ask operations. */ def askTimeout(conf: SparkConf): FiniteDuration = { - Duration.create(conf.getOrElse("spark.akka.askTimeout", "30").toLong, "seconds") + Duration.create(conf.getOrElse("spark.akka.askTimeout", "30").toLong, "seconds") } } diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index bf71d17a21..431d88838f 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -18,16 +18,21 @@ package org.apache.spark.util import java.util.{TimerTask, Timer} -import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.{SparkConf, SparkContext, Logging} /** * Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries) */ -class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, cleanupFunc: (Long) => Unit) extends Logging { +class MetadataCleaner( + cleanerType: MetadataCleanerType.MetadataCleanerType, + cleanupFunc: (Long) => Unit, + conf: SparkConf) + extends Logging +{ val name = cleanerType.toString - private val delaySeconds = MetadataCleaner.getDelaySeconds + private val delaySeconds = MetadataCleaner.getDelaySeconds(conf) private val periodSeconds = math.max(10, delaySeconds / 10) private val timer = new Timer(name + " cleanup timer", true) @@ -65,22 +70,28 @@ object MetadataCleanerType extends Enumeration { def systemProperty(which: MetadataCleanerType.MetadataCleanerType) = "spark.cleaner.ttl." + which.toString } +// TODO: This mutates a Conf to set properties right now, which is kind of ugly when used in the +// initialization of StreamingContext. It's okay for users trying to configure stuff themselves. object MetadataCleaner { - private val conf = SparkContext.globalConf - // using only sys props for now : so that workers can also get to it while preserving earlier behavior. - def getDelaySeconds = conf.getOrElse("spark.cleaner.ttl", "3500").toInt //TODO: this is to fix tests for time being + def getDelaySeconds(conf: SparkConf) = { + conf.getOrElse("spark.cleaner.ttl", "3500").toInt + } - def getDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType): Int = { - conf.getOrElse(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds.toString).toInt + def getDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType): Int = + { + conf.getOrElse(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString) + .toInt } - def setDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType, delay: Int) { + def setDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType, + delay: Int) + { conf.set(MetadataCleanerType.systemProperty(cleanerType), delay.toString) } - def setDelaySeconds(delay: Int, resetAll: Boolean = true) { + def setDelaySeconds(conf: SparkConf, delay: Int, resetAll: Boolean = true) { // override for all ? - conf.set("spark.cleaner.ttl", delay.toString) + conf.set("spark.cleaner.ttl", delay.toString) if (resetAll) { for (cleanerType <- MetadataCleanerType.values) { System.clearProperty(MetadataCleanerType.systemProperty(cleanerType)) diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index 1407c39bfb..bddb3bb735 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -30,10 +30,10 @@ import java.lang.management.ManagementFactory import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.ints.IntOpenHashSet -import org.apache.spark.{SparkConf, SparkContext, Logging} +import org.apache.spark.{SparkEnv, SparkConf, SparkContext, Logging} /** - * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in + * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in * memory-aware caches. * * Based on the following JavaWorld article: @@ -41,7 +41,6 @@ import org.apache.spark.{SparkConf, SparkContext, Logging} */ private[spark] object SizeEstimator extends Logging { - private def conf = SparkContext.globalConf // Sizes of primitive types private val BYTE_SIZE = 1 private val BOOLEAN_SIZE = 1 @@ -90,9 +89,11 @@ private[spark] object SizeEstimator extends Logging { classInfos.put(classOf[Object], new ClassInfo(objectSize, Nil)) } - private def getIsCompressedOops : Boolean = { - if (conf.getOrElse("spark.test.useCompressedOops", null) != null) { - return conf.get("spark.test.useCompressedOops").toBoolean + private def getIsCompressedOops: Boolean = { + // This is only used by tests to override the detection of compressed oops. The test + // actually uses a system property instead of a SparkConf, so we'll stick with that. + if (System.getProperty("spark.test.useCompressedOops") != null) { + return System.getProperty("spark.test.useCompressedOops").toBoolean } try { @@ -104,7 +105,7 @@ private[spark] object SizeEstimator extends Logging { val getVMMethod = hotSpotMBeanClass.getDeclaredMethod("getVMOption", Class.forName("java.lang.String")) - val bean = ManagementFactory.newPlatformMXBeanProxy(server, + val bean = ManagementFactory.newPlatformMXBeanProxy(server, hotSpotMBeanName, hotSpotMBeanClass) // TODO: We could use reflection on the VMOption returned ? return getVMMethod.invoke(bean, "UseCompressedOops").toString.contains("true") @@ -252,7 +253,7 @@ private[spark] object SizeEstimator extends Logging { if (info != null) { return info } - + val parent = getClassInfo(cls.getSuperclass) var shellSize = parent.shellSize var pointerFields = parent.pointerFields diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index fd5888e525..b6b89cc7bb 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -36,15 +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.{SparkContext, SparkException, Logging} +import org.apache.spark.{SparkConf, SparkContext, SparkException, Logging} /** * Various utility methods used by Spark. */ private[spark] object Utils extends Logging { - - private lazy val conf = SparkContext.globalConf /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -240,9 +238,9 @@ private[spark] object Utils extends Logging { * Throws SparkException if the target file already exists and has different contents than * the requested file. */ - def fetchFile(url: String, targetDir: File) { + def fetchFile(url: String, targetDir: File, conf: SparkConf) { val filename = url.split("/").last - val tempDir = getLocalDir + val tempDir = getLocalDir(conf) val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir)) val targetFile = new File(targetDir, filename) val uri = new URI(url) @@ -312,7 +310,7 @@ private[spark] object Utils extends Logging { * return a single directory, even though the spark.local.dir property might be a list of * multiple paths. */ - def getLocalDir: String = { + def getLocalDir(conf: SparkConf): String = { conf.getOrElse("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) } @@ -398,7 +396,7 @@ private[spark] object Utils extends Logging { InetAddress.getByName(address).getHostName } - def localHostPort(): String = { + def localHostPort(conf: SparkConf): String = { val retval = conf.getOrElse("spark.hostPort", null) if (retval == null) { logErrorWithStack("spark.hostPort not set but invoking localHostPort") @@ -838,7 +836,7 @@ private[spark] object Utils extends Logging { } } - /** + /** * Timing method based on iterations that permit JVM JIT optimization. * @param numIters number of iterations * @param f function to be executed 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 ab81bfbe55..8d7546085f 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -20,9 +20,11 @@ package org.apache.spark.io import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import org.scalatest.FunSuite +import org.apache.spark.SparkConf class CompressionCodecSuite extends FunSuite { + val conf = new SparkConf(false) def testCodec(codec: CompressionCodec) { // Write 1000 integers to the output stream, compressed. @@ -43,19 +45,19 @@ class CompressionCodecSuite extends FunSuite { } test("default compression codec") { - val codec = CompressionCodec.createCodec() + val codec = CompressionCodec.createCodec(conf) assert(codec.getClass === classOf[LZFCompressionCodec]) testCodec(codec) } test("lzf compression codec") { - val codec = CompressionCodec.createCodec(classOf[LZFCompressionCodec].getName) + val codec = CompressionCodec.createCodec(conf, classOf[LZFCompressionCodec].getName) assert(codec.getClass === classOf[LZFCompressionCodec]) testCodec(codec) } test("snappy compression codec") { - val codec = CompressionCodec.createCodec(classOf[SnappyCompressionCodec].getName) + val codec = CompressionCodec.createCodec(conf, classOf[SnappyCompressionCodec].getName) assert(codec.getClass === classOf[SnappyCompressionCodec]) testCodec(codec) } 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 2bb827c022..3711382f2e 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 @@ -82,7 +82,7 @@ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { import TaskLocality.{ANY, PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL} private val conf = new SparkConf - val LOCALITY_WAIT = conf.getOrElse("spark.locality.wait", "3000").toLong + val LOCALITY_WAIT = conf.getOrElse("spark.locality.wait", "3000").toLong test("TaskSet with no preferences") { sc = new SparkContext("local", "test") diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index c016c51171..33b0148896 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -22,12 +22,14 @@ import scala.collection.mutable import com.esotericsoftware.kryo.Kryo import org.scalatest.FunSuite -import org.apache.spark.SharedSparkContext +import org.apache.spark.{SparkConf, SharedSparkContext} import org.apache.spark.serializer.KryoTest._ class KryoSerializerSuite extends FunSuite with SharedSparkContext { + val conf = new SparkConf(false) + test("basic types") { - val ser = (new KryoSerializer).newInstance() + val ser = new KryoSerializer(conf).newInstance() def check[T](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } @@ -57,7 +59,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } test("pairs") { - val ser = (new KryoSerializer).newInstance() + val ser = new KryoSerializer(conf).newInstance() def check[T](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } @@ -81,7 +83,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } test("Scala data structures") { - val ser = (new KryoSerializer).newInstance() + val ser = new KryoSerializer(conf).newInstance() def check[T](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } @@ -104,7 +106,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } test("ranges") { - val ser = (new KryoSerializer).newInstance() + val ser = new KryoSerializer(conf).newInstance() def check[T](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) // Check that very long ranges don't get written one element at a time @@ -127,7 +129,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("custom registrator") { System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) - val ser = (new KryoSerializer).newInstance() + val ser = new KryoSerializer(conf).newInstance() def check[T](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } 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 4ef5538951..a0fc3445be 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.{SparkConf, SparkContext} class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { - private val conf = new SparkConf + private val conf = new SparkConf(false) var store: BlockManager = null var store2: BlockManager = null var actorSystem: ActorSystem = null @@ -45,7 +45,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test conf.set("spark.kryoserializer.buffer.mb", "1") - val serializer = new KryoSerializer + val serializer = new KryoSerializer(conf) // Implicitly convert strings to BlockIds for test clarity. implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) @@ -167,7 +167,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("master + 2 managers interaction") { store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf) - store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer, 2000, conf) + store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf) val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -654,7 +654,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block store put failure") { // Use Java serializer so we can create an unserializable error. - store = new BlockManager("", actorSystem, master, new JavaSerializer, 1200, conf) + store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf) // The put should fail since a1 is not serializable. class UnserializableClass diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index a5facd5bbd..11ebdc352b 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -140,8 +140,6 @@ class SizeEstimatorSuite test("64-bit arch with no compressed oops") { val arch = System.setProperty("os.arch", "amd64") val oops = System.setProperty("spark.test.useCompressedOops", "false") - SparkContext.globalConf.set("os.arch", "amd64") - SparkContext.globalConf.set("spark.test.useCompressedOops", "false") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() 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 12c430be27..4c0de46964 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) } val sparkConf = new SparkConf() - sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") sparkConf.set("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) val inputFile = args(0) @@ -46,7 +46,7 @@ object WikipediaPageRank { val host = args(3) val usePartitioner = args(4).toBoolean - sparkConf.setMasterUrl(host).setAppName("WikipediaPageRank") + sparkConf.setMaster(host).setAppName("WikipediaPageRank") val sc = new SparkContext(sparkConf) // Parse the Wikipedia page data into a graph 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 5bf0b7a24a..2cf273a702 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 @@ -35,7 +35,7 @@ object WikipediaPageRankStandalone { System.exit(-1) } val sparkConf = new SparkConf() - sparkConf.set("spark.serializer", "spark.bagel.examples.WPRSerializer") + sparkConf.set("spark.serializer", "spark.bagel.examples.WPRSerializer") val inputFile = args(0) @@ -44,7 +44,7 @@ object WikipediaPageRankStandalone { val host = args(3) val usePartitioner = args(4).toBoolean - sparkConf.setMasterUrl(host).setAppName("WikipediaPageRankStandalone") + sparkConf.setMaster(host).setAppName("WikipediaPageRankStandalone") val sc = new SparkContext(sparkConf) 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 2f2d106f86..8b27ecf82c 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 @@ -579,12 +579,12 @@ object ALS { val alpha = if (args.length >= 8) args(7).toDouble else 1 val blocks = if (args.length == 9) args(8).toInt else -1 val sc = new SparkContext(master, "ALS") - sc.conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + sc.conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") sc.conf.set("spark.kryo.registrator", classOf[ALSRegistrator].getName) - sc.conf.set("spark.kryo.referenceTracking", "false") - sc.conf.set("spark.kryoserializer.buffer.mb", "8") - sc.conf.set("spark.locality.wait", "10000") - + sc.conf.set("spark.kryo.referenceTracking", "false") + sc.conf.set("spark.kryoserializer.buffer.mb", "8") + sc.conf.set("spark.locality.wait", "10000") + val ratings = sc.textFile(ratingsFile).map { line => val fields = line.split(',') Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 433268a1dd..91e35e2d34 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.Utils class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging { def this(args: ApplicationMasterArguments) = this(args, new Configuration()) - + private var rpc: YarnRPC = YarnRPC.create(conf) private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private var appAttemptId: ApplicationAttemptId = _ @@ -81,12 +81,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Workaround until hadoop moves to something which has // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line) // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf) - + ApplicationMaster.register(this) // Start the user's JAR userThread = startUserClass() - + // This a bit hacky, but we need to wait until the spark.driver.port property has // been set by the Thread executing the user class. waitForSparkMaster() @@ -99,7 +99,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Allocate all containers allocateWorkers() - // Wait for the user class to Finish + // Wait for the user class to Finish userThread.join() System.exit(0) @@ -119,7 +119,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } localDirs } - + private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name()) @@ -128,17 +128,17 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("ApplicationAttemptId: " + appAttemptId) appAttemptId } - + private def registerApplicationMaster(): RegisterApplicationMasterResponse = { logInfo("Registering the ApplicationMaster") amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) } - + private def waitForSparkMaster() { logInfo("Waiting for Spark driver to be reachable.") var driverUp = false var tries = 0 - val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt while (!driverUp && tries < numTries) { val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") @@ -199,7 +199,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e ApplicationMaster.sparkContextRef.synchronized { var numTries = 0 val waitTime = 10000L - val maxNumTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt + val maxNumTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries) { logInfo("Waiting for Spark context initialization ... " + numTries) numTries = numTries + 1 @@ -214,7 +214,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e yarnConf, amClient, appAttemptId, - args, + args, sparkContext.preferredNodeLocationData) } else { logWarning("Unable to retrieve SparkContext inspite of waiting for %d, maxNumTries = %d". @@ -265,7 +265,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) @@ -314,11 +314,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e for (container <- containers) { logInfo("Launching shell command on a new container." + ", containerId=" + container.getId() - + ", containerNode=" + container.getNodeId().getHost() + + ", containerNode=" + container.getNodeId().getHost() + ":" + container.getNodeId().getPort() + ", containerNodeURI=" + container.getNodeHttpAddress() + ", containerState" + container.getState() - + ", containerResourceMemory" + + ", containerResourceMemory" + container.getResource().getMemory()) } } @@ -338,12 +338,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } /** - * Clean up the staging directory. + * Clean up the staging directory. */ - private def cleanupStagingDir() { + private def cleanupStagingDir() { var stagingDirPath: Path = null try { - val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { @@ -359,7 +359,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } } - // The shutdown hook that runs when a signal is received AND during normal close of the JVM. + // The shutdown hook that runs when a signal is received AND during normal close of the JVM. class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable { def run() { @@ -415,18 +415,18 @@ object ApplicationMaster { // Note that this will unfortunately not properly clean up the staging files because it gets // called too late, after the filesystem is already shutdown. if (modified) { - Runtime.getRuntime().addShutdownHook(new Thread with Logging { + Runtime.getRuntime().addShutdownHook(new Thread with Logging { // This is not only logs, but also ensures that log system is initialized for this instance // when we are actually 'run'-ing. logInfo("Adding shutdown hook for context " + sc) - override def run() { - logInfo("Invoking sc stop from shutdown hook") - sc.stop() + override def run() { + logInfo("Invoking sc stop from shutdown hook") + sc.stop() // Best case ... for (master <- applicationMasters) { master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) } - } + } } ) } diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index a322f60864..963b5b88be 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, Records} -import org.apache.spark.Logging +import org.apache.spark.Logging import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil @@ -150,7 +150,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl queueInfo.getChildQueues.size)) } - def verifyClusterResources(app: GetNewApplicationResponse) = { + def verifyClusterResources(app: GetNewApplicationResponse) = { val maxMem = app.getMaximumResourceCapability().getMemory() logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) @@ -221,7 +221,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf) fs.setReplication(newPath, replication) if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION)) - } + } // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific // version shows the specific version in the distributed cache configuration val qualPath = fs.makeQualified(newPath) @@ -244,7 +244,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort + val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort if (UserGroupInformation.isSecurityEnabled()) { val dstFs = dst.getFileSystem(conf) @@ -269,7 +269,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } val setPermissions = if (destName.equals(Client.APP_JAR)) true else false val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, destName, statCache) } } @@ -283,7 +283,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val destPath = copyRemoteFile(dst, localPath, replication) // Only add the resource to the Spark ApplicationMaster. val appMasterOnly = true - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, linkname, statCache, appMasterOnly) } } @@ -295,7 +295,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) val destPath = copyRemoteFile(dst, localPath, replication) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, linkname, statCache) } } @@ -307,7 +307,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) val destPath = copyRemoteFile(dst, localPath, replication) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, linkname, statCache) } } @@ -317,7 +317,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } def setupLaunchEnv( - localResources: HashMap[String, LocalResource], + localResources: HashMap[String, LocalResource], stagingDir: String): HashMap[String, String] = { logInfo("Setting up the launch environment") val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null) @@ -406,11 +406,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } val commands = List[String]( - javaCommand + + javaCommand + " -server " + JAVA_OPTS + " " + args.amClass + - " --class " + args.userClass + + " --class " + args.userClass + " --jar " + args.userJar + userArgsToString(args) + " --worker-memory " + args.workerMemory + @@ -436,7 +436,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl super.submitApplication(appContext) } - def monitorApplication(appId: ApplicationId): Boolean = { + def monitorApplication(appId: ApplicationId): Boolean = { while (true) { Thread.sleep(1000) val report = super.getApplicationReport(appId) @@ -458,7 +458,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val state = report.getYarnApplicationState() val dsStatus = report.getFinalApplicationStatus() - if (state == YarnApplicationState.FINISHED || + if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) { return true @@ -495,25 +495,25 @@ object Client { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$()) // If log4j present, ensure ours overrides all others if (addLog4j) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + LOG4J_PROP) } // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") + val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") .toBoolean if (userClasspathFirst) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + APP_JAR) } - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + SPARK_JAR) Client.populateHadoopClasspath(conf, env) if (!userClasspathFirst) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + APP_JAR) } - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + "*") } } diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 41ac292249..1a9bb97b3e 100644 --- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -35,7 +35,7 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 // MB var workerCores = 1 var numWorkers = 2 - var amQueue = conf.getOrElse("QUEUE", "default") + var amQueue = conf.getOrElse("QUEUE", "default") var amMemory: Int = 512 // MB var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index b2f499e637..f108c70f21 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -35,6 +35,7 @@ import java.lang.{Class => jClass} import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse} import org.apache.spark.Logging +import org.apache.spark.SparkConf import org.apache.spark.SparkContext /** The Scala interactive shell. It provides a read-eval-print loop @@ -929,7 +930,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } def createSparkContext(): SparkContext = { - val uri = System.getenv("SPARK_EXECUTOR_URI") + val execUri = System.getenv("SPARK_EXECUTOR_URI") val master = this.master match { case Some(m) => m case None => { @@ -938,11 +939,16 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } val jars = SparkILoop.getAddedJars.map(new java.io.File(_).getAbsolutePath) - sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars) - if (uri != null) { - sparkContext.conf.set("spark.executor.uri", uri) + val conf = new SparkConf() + .setMaster(master) + .setAppName("Spark shell") + .setSparkHome(System.getenv("SPARK_HOME")) + .setJars(jars) + .set("spark.repl.class.uri", intp.classServer.uri) + if (execUri != null) { + conf.set("spark.executor.uri", execUri) } - sparkContext.conf.set("spark.repl.class.uri", intp.classServer.uri) + sparkContext = new SparkContext(conf) echo("Created spark context..") sparkContext } 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 0d412e4478..a993083164 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -34,7 +34,7 @@ import scala.tools.reflect.StdRuntimeTags._ import scala.util.control.ControlThrowable import util.stackTraceString -import org.apache.spark.{SparkContext, HttpServer, SparkEnv, Logging} +import org.apache.spark.{HttpServer, SparkConf, Logging} import org.apache.spark.util.Utils // /** directory to save .class files to */ @@ -89,7 +89,7 @@ import org.apache.spark.util.Utils /** Local directory to save .class files too */ val outputDir = { val tmp = System.getProperty("java.io.tmpdir") - val rootDir = SparkContext.globalConf.getOrElse("spark.repl.classdir", tmp) + val rootDir = new SparkConf().getOrElse("spark.repl.classdir", tmp) Utils.createTempDir(rootDir) } if (SPARK_DEBUG_REPL) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index b8e1427a21..f106bba678 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -24,7 +24,7 @@ import java.util.concurrent.RejectedExecutionException import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.io.CompressionCodec import org.apache.spark.util.MetadataCleaner @@ -36,12 +36,11 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) val framework = ssc.sc.appName val sparkHome = ssc.sc.getSparkHome.getOrElse(null) val jars = ssc.sc.jars - val environment = ssc.sc.environment val graph = ssc.graph val checkpointDir = ssc.checkpointDir val checkpointDuration = ssc.checkpointDuration val pendingTimes = ssc.scheduler.jobManager.getPendingTimes() - val delaySeconds = MetadataCleaner.getDelaySeconds + val delaySeconds = MetadataCleaner.getDelaySeconds(ssc.conf) val sparkConf = ssc.sc.conf def validate() { @@ -58,7 +57,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) * Convenience class to speed up the writing of graph checkpoint to file */ private[streaming] -class CheckpointWriter(checkpointDir: String) extends Logging { +class CheckpointWriter(conf: SparkConf, checkpointDir: String) extends Logging { val file = new Path(checkpointDir, "graph") // The file to which we actually write - and then "move" to file. private val writeFile = new Path(file.getParent, file.getName + ".next") @@ -66,14 +65,14 @@ class CheckpointWriter(checkpointDir: String) extends Logging { private var stopped = false - val conf = new Configuration() - var fs = file.getFileSystem(conf) + val hadoopConf = new Configuration() + var fs = file.getFileSystem(hadoopConf) val maxAttempts = 3 val executor = Executors.newFixedThreadPool(1) - private val compressionCodec = CompressionCodec.createCodec() + private val compressionCodec = CompressionCodec.createCodec(conf) - // Removed code which validates whether there is only one CheckpointWriter per path 'file' since + // Removed code which validates whether there is only one CheckpointWriter per path 'file' since // I did not notice any errors - reintroduce it ? class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable { @@ -142,11 +141,12 @@ class CheckpointWriter(checkpointDir: String) extends Logging { private[streaming] object CheckpointReader extends Logging { - def read(path: String): Checkpoint = { + def read(conf: SparkConf, path: String): Checkpoint = { val fs = new Path(path).getFileSystem(new Configuration()) - val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk"), new Path(path), new Path(path + ".bk")) + val attempts = Seq( + new Path(path, "graph"), new Path(path, "graph.bk"), new Path(path), new Path(path + ".bk")) - val compressionCodec = CompressionCodec.createCodec() + val compressionCodec = CompressionCodec.createCodec(conf) attempts.foreach(file => { if (fs.exists(file)) { 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 329d2b5835..8005202500 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala @@ -213,7 +213,7 @@ abstract class DStream[T: ClassTag] ( checkpointDuration + "). Please set it to higher than " + checkpointDuration + "." ) - val metadataCleanerDelay = MetadataCleaner.getDelaySeconds + val metadataCleanerDelay = MetadataCleaner.getDelaySeconds(ssc.conf) logInfo("metadataCleanupDelay = " + metadataCleanerDelay) assert( metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000, diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala index 1d23713c80..82ed6bed69 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala @@ -26,10 +26,10 @@ class Scheduler(ssc: StreamingContext) extends Logging { initLogging() - val concurrentJobs = ssc.sc.conf.getOrElse("spark.streaming.concurrentJobs", "1").toInt + val concurrentJobs = ssc.sc.conf.getOrElse("spark.streaming.concurrentJobs", "1").toInt val jobManager = new JobManager(ssc, concurrentJobs) val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) { - new CheckpointWriter(ssc.checkpointDir) + new CheckpointWriter(ssc.conf, ssc.checkpointDir) } else { null } @@ -50,13 +50,13 @@ class Scheduler(ssc: StreamingContext) extends Logging { } logInfo("Scheduler started") } - + def stop() = synchronized { timer.stop() jobManager.stop() if (checkpointWriter != null) checkpointWriter.stop() ssc.graph.stop() - logInfo("Scheduler stopped") + logInfo("Scheduler stopped") } private def startFirstTime() { @@ -73,7 +73,7 @@ class Scheduler(ssc: StreamingContext) extends Logging { // or if the property is defined set it to that time if (clock.isInstanceOf[ManualClock]) { val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds - val jumpTime = ssc.sc.conf.getOrElse("spark.streaming.manualClock.jump", "0").toLong + val jumpTime = ssc.sc.conf.getOrElse("spark.streaming.manualClock.jump", "0").toLong clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime) } 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 76744223e1..079841ad9d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -87,13 +87,12 @@ class StreamingContext private ( null, batchDuration) } - /** * Re-create a StreamingContext from a checkpoint file. * @param path Path either to the directory that was specified as the checkpoint directory, or * to the checkpoint file 'graph' or 'graph.bk'. */ - def this(path: String) = this(null, CheckpointReader.read(path), null) + def this(path: String) = this(null, CheckpointReader.read(new SparkConf(), path), null) initLogging() @@ -102,11 +101,13 @@ class StreamingContext private ( "both SparkContext and checkpoint as null") } - if(cp_ != null && cp_.delaySeconds >= 0 && MetadataCleaner.getDelaySeconds < 0) { - MetadataCleaner.setDelaySeconds(cp_.delaySeconds) + private val conf_ = Option(sc_).map(_.conf).getOrElse(cp_.sparkConf) + + if(cp_ != null && cp_.delaySeconds >= 0 && MetadataCleaner.getDelaySeconds(conf_) < 0) { + MetadataCleaner.setDelaySeconds(conf_, cp_.delaySeconds) } - if (MetadataCleaner.getDelaySeconds < 0) { + if (MetadataCleaner.getDelaySeconds(conf_) < 0) { throw new SparkException("Spark Streaming cannot be used without setting spark.cleaner.ttl; " + "set this property before creating a SparkContext (use SPARK_JAVA_OPTS for the shell)") } @@ -115,12 +116,14 @@ class StreamingContext private ( protected[streaming] val sc: SparkContext = { if (isCheckpointPresent) { - new SparkContext(cp_.sparkConf, cp_.environment) + new SparkContext(cp_.sparkConf) } else { sc_ } } + protected[streaming] val conf = sc.conf + protected[streaming] val env = SparkEnv.get protected[streaming] val graph: DStreamGraph = { @@ -579,13 +582,15 @@ object StreamingContext { appName: String, sparkHome: String, jars: Seq[String], - environment: Map[String, String]): SparkContext = { + environment: Map[String, String]): SparkContext = + { + val sc = new SparkContext(master, appName, sparkHome, jars, environment) // Set the default cleaner delay to an hour if not already set. // This should be sufficient for even 1 second interval. - if (MetadataCleaner.getDelaySeconds < 0) { - MetadataCleaner.setDelaySeconds(3600) + if (MetadataCleaner.getDelaySeconds(sc.conf) < 0) { + MetadataCleaner.setDelaySeconds(sc.conf, 3600) } - new SparkContext(master, appName, sparkHome, jars, environment) + sc } protected[streaming] def rddToFileName[T](prefix: String, suffix: String, time: Time): String = { 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 8bf761b8cb..bd607f9d18 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 @@ -175,8 +175,8 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging /** A helper actor that communicates with the NetworkInputTracker */ private class NetworkReceiverActor extends Actor { logInfo("Attempting to register with tracker") - val ip = env.conf.getOrElse("spark.driver.host", "localhost") - val port = env.conf.getOrElse("spark.driver.port", "7077").toInt + val ip = env.conf.getOrElse("spark.driver.host", "localhost") + val port = env.conf.getOrElse("spark.driver.port", "7077").toInt val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) val tracker = env.actorSystem.actorSelection(url) val timeout = 5.seconds @@ -213,7 +213,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null) val clock = new SystemClock() - val blockInterval = env.conf.getOrElse("spark.streaming.blockInterval", "200").toLong + val blockInterval = env.conf.getOrElse("spark.streaming.blockInterval", "200").toLong val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer) val blockStorageLevel = storageLevel val blocksForPushing = new ArrayBlockingQueue[Block](1000) 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 fc8655a083..6585d494a6 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,7 +20,7 @@ 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} +import org.apache.spark.{SparkConf, Logging} import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream import scala.io.Source import java.io.IOException @@ -42,7 +42,7 @@ object RawTextSender extends Logging { // Repeat the input data multiple times to fill in a buffer val lines = Source.fromFile(file).getLines().toArray val bufferStream = new FastByteArrayOutputStream(blockSize + 1000) - val ser = new KryoSerializer().newInstance() + val ser = new KryoSerializer(new SparkConf()).newInstance() val serStream = ser.serializeStream(bufferStream) var i = 0 while (bufferStream.position < blockSize) { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index da8f135dd7..8c16daa21c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -52,9 +52,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { override def checkpointDir = "checkpoint" - before { - conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") - } + conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") after { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown @@ -70,7 +68,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams val ssc = new StreamingContext(new SparkContext(conf), batchDuration) val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String ]] + val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] val outputStream = new TestOutputStream(networkStream, outputBuffer) def output = outputBuffer.flatMap(x => x) ssc.registerOutputStream(outputStream) 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 d1cab0c609..a265284bff 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -130,7 +130,11 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Whether to actually wait in real time before changing manual clock def actuallyWait = false - def conf = new SparkConf().setMasterUrl(master).setAppName(framework).set("spark.cleaner.ttl", "3600") + val conf = new SparkConf() + .setMaster(master) + .setAppName(framework) + .set("spark.cleaner.ttl", "3600") + /** * Set up required DStreams to test the DStream operation using the two sequences * of input collections. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 1dd38dd13e..dc9228180f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -43,7 +43,7 @@ import org.apache.spark.util.Utils class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging { def this(args: ApplicationMasterArguments) = this(args, new Configuration()) - + private var rpc: YarnRPC = YarnRPC.create(conf) private var resourceManager: AMRMProtocol = _ private var appAttemptId: ApplicationAttemptId = _ @@ -68,7 +68,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) - + appAttemptId = getApplicationAttemptId() isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts resourceManager = registerWithResourceManager() @@ -92,11 +92,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // } //} // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf) - + ApplicationMaster.register(this) // Start the user's JAR userThread = startUserClass() - + // This a bit hacky, but we need to wait until the spark.driver.port property has // been set by the Thread executing the user class. waitForSparkMaster() @@ -105,11 +105,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Do this after spark master is up and SparkContext is created so that we can register UI Url val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() - + // Allocate all containers allocateWorkers() - - // Wait for the user class to Finish + + // Wait for the user class to Finish userThread.join() System.exit(0) @@ -129,7 +129,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } localDirs } - + private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) @@ -138,7 +138,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("ApplicationAttemptId: " + appAttemptId) appAttemptId } - + private def registerWithResourceManager(): AMRMProtocol = { val rmAddress = NetUtils.createSocketAddr(yarnConf.get( YarnConfiguration.RM_SCHEDULER_ADDRESS, @@ -146,26 +146,26 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("Connecting to ResourceManager at " + rmAddress) rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] } - + private def registerApplicationMaster(): RegisterApplicationMasterResponse = { logInfo("Registering the ApplicationMaster") val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest]) .asInstanceOf[RegisterApplicationMasterRequest] appMasterRequest.setApplicationAttemptId(appAttemptId) // Setting this to master host,port - so that the ApplicationReport at client has some - // sensible info. + // sensible info. // Users can then monitor stderr/stdout on that node if required. appMasterRequest.setHost(Utils.localHostName()) appMasterRequest.setRpcPort(0) appMasterRequest.setTrackingUrl(uiAddress) resourceManager.registerApplicationMaster(appMasterRequest) } - + private def waitForSparkMaster() { logInfo("Waiting for spark driver to be reachable.") var driverUp = false var tries = 0 - val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.applicationMaster.waitTries", "10").toInt while(!driverUp && tries < numTries) { val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") @@ -226,7 +226,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e ApplicationMaster.sparkContextRef.synchronized { var count = 0 val waitTime = 10000L - val numTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt + val numTries = conf.getOrElse("spark.yarn.ApplicationMaster.waitTries", "10").toInt while (ApplicationMaster.sparkContextRef.get() == null && count < numTries) { logInfo("Waiting for spark context initialization ... " + count) count = count + 1 @@ -241,8 +241,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e yarnConf, resourceManager, appAttemptId, - args, - sparkContext.preferredNodeLocationData) + args, + sparkContext.preferredNodeLocationData) } else { logWarning("Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d". format(count * waitTime, numTries)) @@ -294,7 +294,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + conf.getOrElse("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) @@ -342,11 +342,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e for (container <- containers) { logInfo("Launching shell command on a new container." + ", containerId=" + container.getId() - + ", containerNode=" + container.getNodeId().getHost() + + ", containerNode=" + container.getNodeId().getHost() + ":" + container.getNodeId().getPort() + ", containerNodeURI=" + container.getNodeHttpAddress() + ", containerState" + container.getState() - + ", containerResourceMemory" + + ", containerResourceMemory" + container.getResource().getMemory()) } } @@ -372,12 +372,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } /** - * Clean up the staging directory. + * Clean up the staging directory. */ - private def cleanupStagingDir() { + private def cleanupStagingDir() { var stagingDirPath: Path = null try { - val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = conf.getOrElse("spark.yarn.preserve.staging.files", "false").toBoolean if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { @@ -393,7 +393,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } } - // The shutdown hook that runs when a signal is received AND during normal close of the JVM. + // The shutdown hook that runs when a signal is received AND during normal close of the JVM. class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable { def run() { @@ -446,18 +446,18 @@ object ApplicationMaster { // Note that this will unfortunately not properly clean up the staging files because it gets // called too late, after the filesystem is already shutdown. if (modified) { - Runtime.getRuntime().addShutdownHook(new Thread with Logging { + Runtime.getRuntime().addShutdownHook(new Thread with Logging { // This is not only logs, but also ensures that log system is initialized for this instance // when we are actually 'run'-ing. logInfo("Adding shutdown hook for context " + sc) - override def run() { - logInfo("Invoking sc stop from shutdown hook") - sc.stop() + override def run() { + logInfo("Invoking sc stop from shutdown hook") + sc.stop() // Best case ... for (master <- applicationMasters) { master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) } - } + } } ) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 29892e98e3..cc150888eb 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, Records} -import org.apache.spark.Logging +import org.apache.spark.Logging import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil @@ -59,7 +59,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short) // App files are world-wide readable and owner writable -> rw-r--r-- - val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) + val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) // for client user who want to monitor app status by itself. def runApp() = { @@ -103,7 +103,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD), (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size " + "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD) - ).foreach { case(cond, errStr) => + ).foreach { case(cond, errStr) => if (cond) { logError(errStr) args.printUsageAndExit(1) @@ -130,7 +130,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl queueInfo.getChildQueues.size)) } - def verifyClusterResources(app: GetNewApplicationResponse) = { + def verifyClusterResources(app: GetNewApplicationResponse) = { val maxMem = app.getMaximumResourceCapability().getMemory() logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) @@ -146,7 +146,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } // We could add checks to make sure the entire cluster has enough resources but that involves - // getting all the node reports and computing ourselves + // getting all the node reports and computing ourselves } def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = { @@ -207,7 +207,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf) fs.setReplication(newPath, replication) if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION)) - } + } // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific // version shows the specific version in the distributed cache configuration val qualPath = fs.makeQualified(newPath) @@ -230,7 +230,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort + val replication = conf.getOrElse("spark.yarn.submit.file.replication", "3").toShort if (UserGroupInformation.isSecurityEnabled()) { val dstFs = dst.getFileSystem(conf) @@ -241,7 +241,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - Map(Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar, + Map(Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar, Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF")) .foreach { case(destName, _localPath) => val localPath: String = if (_localPath != null) _localPath.trim() else "" @@ -253,7 +253,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } val setPermissions = if (destName.equals(Client.APP_JAR)) true else false val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, destName, statCache) } } @@ -265,7 +265,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) val destPath = copyRemoteFile(dst, localPath, replication) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, linkname, statCache, true) } } @@ -277,7 +277,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) val destPath = copyRemoteFile(dst, localPath, replication) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, linkname, statCache) } } @@ -289,7 +289,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) val destPath = copyRemoteFile(dst, localPath, replication) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, linkname, statCache) } } @@ -299,7 +299,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } def setupLaunchEnv( - localResources: HashMap[String, LocalResource], + localResources: HashMap[String, LocalResource], stagingDir: String): HashMap[String, String] = { logInfo("Setting up the launch environment") val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null) @@ -354,7 +354,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // Add Xmx for am memory JAVA_OPTS += "-Xmx" + amMemory + "m " - JAVA_OPTS += " -Djava.io.tmpdir=" + + 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 @@ -387,11 +387,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl javaCommand = Environment.JAVA_HOME.$() + "/bin/java" } - val commands = List[String](javaCommand + + val commands = List[String](javaCommand + " -server " + JAVA_OPTS + " " + args.amClass + - " --class " + args.userClass + + " --class " + args.userClass + " --jar " + args.userJar + userArgsToString(args) + " --worker-memory " + args.workerMemory + @@ -421,7 +421,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl super.submitApplication(appContext) } - def monitorApplication(appId: ApplicationId): Boolean = { + def monitorApplication(appId: ApplicationId): Boolean = { while (true) { Thread.sleep(1000) val report = super.getApplicationReport(appId) @@ -443,7 +443,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val state = report.getYarnApplicationState() val dsStatus = report.getFinalApplicationStatus() - if (state == YarnApplicationState.FINISHED || + if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) { return true @@ -461,7 +461,7 @@ 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 - conf.set("SPARK_YARN_MODE", "true") + System.setProperty("SPARK_YARN_MODE", "true") val args = new ClientArguments(argStrings) @@ -479,25 +479,25 @@ object Client { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$()) // If log4j present, ensure ours overrides all others if (addLog4j) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + LOG4J_PROP) } // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") + val userClasspathFirst = conf.getOrElse("spark.yarn.user.classpath.first", "false") .toBoolean if (userClasspathFirst) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + APP_JAR) } - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + SPARK_JAR) Client.populateHadoopClasspath(conf, env) if (!userClasspathFirst) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + APP_JAR) } - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + "*") } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 617289f568..e9e46a193b 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -33,7 +33,7 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 var workerCores = 1 var numWorkers = 2 - var amQueue = conf.getOrElse("QUEUE", "default") + var amQueue = conf.getOrElse("QUEUE", "default") var amMemory: Int = 512 var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" -- cgit v1.2.3 From a16c52ed1bf5b416c9ec8c13dbe84ee203032b1b Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 28 Dec 2013 17:24:21 -0500 Subject: Check for SPARK_YARN_MODE through a system property too since it can sometimes be set that way (undoes a change in previous commit) --- core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 4f402c1121..27dc42bf7e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -69,7 +69,8 @@ class SparkHadoopUtil { object SparkHadoopUtil { private val hadoop = { - val yarnMode = java.lang.Boolean.valueOf(System.getenv("SPARK_YARN_MODE")) + val yarnMode = java.lang.Boolean.valueOf( + System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) if (yarnMode) { try { Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil") -- cgit v1.2.3 From 5bbe73864eea78b76448ce42a7af847dad73b269 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 28 Dec 2013 17:31:58 -0500 Subject: Fix Executor not getting properties in local mode --- core/src/main/scala/org/apache/spark/SparkConf.scala | 2 +- .../main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 185ddb1fe5..bd24cd19f2 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -28,7 +28,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { if (loadDefaults) { val typesafeConfig = ConfigFactory.systemProperties() .withFallback(ConfigFactory.parseResources("spark.conf")) - for (e <- typesafeConfig.entrySet().asScala) { + for (e <- typesafeConfig.entrySet().asScala if e.getKey.startsWith("spark.")) { settings(e.getKey) = e.getValue.unwrapped.toString } } 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 8498cffd31..7c173e3ad5 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 @@ -50,7 +50,8 @@ private[spark] class LocalActor(localScheduler: LocalScheduler, private var freeCores: Int) extends Actor with Logging { - val executor = new Executor("localhost", "localhost", Seq.empty, isLocal = true) + val executor = new Executor( + "localhost", "localhost", localScheduler.sc.conf.getAll, isLocal = true) def receive = { case LocalReviveOffers => -- cgit v1.2.3 From 20631348d198ba52059f278c1b415c3a80a95b81 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 28 Dec 2013 23:17:58 -0500 Subject: Fix other failing tests --- .../scala/org/apache/spark/io/CompressionCodec.scala | 2 +- .../scala/org/apache/spark/SharedSparkContext.scala | 4 +++- .../org/apache/spark/metrics/MetricsSystemSuite.scala | 4 ++-- .../apache/spark/serializer/KryoSerializerSuite.scala | 17 ++--------------- .../org/apache/spark/streaming/StreamingContext.scala | 14 +++++++------- .../spark/streaming/api/java/JavaStreamingContext.scala | 9 +++++++++ .../java/org/apache/spark/streaming/JavaAPISuite.java | 9 ++++++--- .../org/apache/spark/streaming/CheckpointSuite.scala | 8 -------- 8 files changed, 30 insertions(+), 37 deletions(-) (limited to 'core/src') 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 20402686a8..075a18b068 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -46,7 +46,7 @@ private[spark] object CompressionCodec { def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { val ctor = Class.forName(codecName, true, Thread.currentThread.getContextClassLoader) .getConstructor(classOf[SparkConf]) - ctor.newInstance(conf).asInstanceOf[CompressionCodec] + ctor.newInstance(conf).asInstanceOf[CompressionCodec] } } diff --git a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala index 288aa14eeb..c650ef4ed5 100644 --- a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala @@ -27,8 +27,10 @@ trait SharedSparkContext extends BeforeAndAfterAll { self: Suite => def sc: SparkContext = _sc + var conf = new SparkConf(false) + override def beforeAll() { - _sc = new SparkContext("local", "test") + _sc = new SparkContext("local", "test", conf) super.beforeAll() } diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index 4ecdde0001..71a2c6c498 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -24,10 +24,10 @@ import org.apache.spark.SparkConf class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ var conf: SparkConf = null + before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() - System.setProperty("spark.metrics.conf", filePath) - conf = new SparkConf + conf = new SparkConf(false).set("spark.metrics.conf", filePath) } test("MetricsSystem with default config") { diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 33b0148896..d23e01418b 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -26,7 +26,8 @@ import org.apache.spark.{SparkConf, SharedSparkContext} import org.apache.spark.serializer.KryoTest._ class KryoSerializerSuite extends FunSuite with SharedSparkContext { - val conf = new SparkConf(false) + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) test("basic types") { val ser = new KryoSerializer(conf).newInstance() @@ -127,8 +128,6 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } test("custom registrator") { - System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) - val ser = new KryoSerializer(conf).newInstance() def check[T](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) @@ -188,18 +187,6 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { .fold(new ClassWithoutNoArgConstructor(10))((t1, t2) => new ClassWithoutNoArgConstructor(t1.x + t2.x)).x assert(10 + control.sum === result) } - - override def beforeAll() { - System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) - super.beforeAll() - } - - override def afterAll() { - super.afterAll() - System.clearProperty("spark.kryo.registrator") - System.clearProperty("spark.serializer") - } } object KryoTest { 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 9d2033fd11..286ec285a9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -63,8 +63,8 @@ class StreamingContext private ( /** * Create a StreamingContext using an existing SparkContext. - * @param sparkContext Existing SparkContext - * @param batchDuration The time interval at which streaming data will be divided into batches + * @param sparkContext existing SparkContext + * @param batchDuration the time interval at which streaming data will be divided into batches */ def this(sparkContext: SparkContext, batchDuration: Duration) = { this(sparkContext, null, batchDuration) @@ -72,8 +72,8 @@ class StreamingContext private ( /** * Create a StreamingContext by providing the configuration necessary for a new SparkContext. - * @param conf A standard Spark application configuration - * @param batchDuration The time interval at which streaming data will be divided into batches + * @param conf a [[org.apache.spark.SparkConf]] object specifying Spark parameters + * @param batchDuration the time interval at which streaming data will be divided into batches */ def this(conf: SparkConf, batchDuration: Duration) = { this(StreamingContext.createNewSparkContext(conf), null, batchDuration) @@ -81,9 +81,9 @@ class StreamingContext private ( /** * Create a StreamingContext by providing the details necessary for creating a new SparkContext. - * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). - * @param appName A name for your job, to display on the cluster web UI - * @param batchDuration The time interval at which streaming data will be divided into batches + * @param master cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName a name for your job, to display on the cluster web UI + * @param batchDuration the time interval at which streaming data will be divided into batches */ def this( master: String, 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 80dcf87491..5842a7cd68 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 @@ -39,6 +39,7 @@ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => J import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaRDD} import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream._ +import org.apache.spark.SparkConf /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -122,6 +123,14 @@ class JavaStreamingContext(val ssc: StreamingContext) { def this(sparkContext: JavaSparkContext, batchDuration: Duration) = this(new StreamingContext(sparkContext.sc, batchDuration)) + /** + * Creates a StreamingContext using an existing SparkContext. + * @param conf A Spark application configuration + * @param batchDuration The time interval at which streaming data will be divided into batches + */ + def this(conf: SparkConf, batchDuration: Duration) = + this(new StreamingContext(conf, batchDuration)) + /** * Re-creates a StreamingContext from a checkpoint file. * @param path Path either to the directory that was specified as the checkpoint directory, or diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index a1db0995e3..d53d433693 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -25,6 +25,7 @@ import com.google.common.io.Files; import kafka.serializer.StringDecoder; import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaDStreamLike; import org.junit.After; import org.junit.Assert; @@ -62,14 +63,16 @@ public class JavaAPISuite implements Serializable { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } @After public void tearDown() { - System.clearProperty("spark.streaming.clock"); ssc.stop(); ssc = null; diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 2a41ec0035..ca230fd056 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -201,10 +201,6 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { // It also tests whether batches, whose processing was incomplete due to the // failure, are re-processed or not. test("recovery with file input stream") { - // Disable manual clock as FileInputDStream does not work with manual clock - val clockProperty = System.getProperty("spark.streaming.clock") - System.clearProperty("spark.streaming.clock") - // Set up the streaming context and input streams val testDir = Files.createTempDir() var ssc = new StreamingContext(master, framework, Seconds(1)) @@ -301,10 +297,6 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { ) // To ensure that all the inputs were received correctly assert(expectedOutput.last === output.last) - - // Enable manual clock back again for other tests - if (clockProperty != null) - System.setProperty("spark.streaming.clock", clockProperty) } -- cgit v1.2.3 From d812aeece9926058a5685fe700fecbe3d31a7fa5 Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Sat, 28 Dec 2013 23:21:49 -0500 Subject: Factor call site reporting out to SparkContext. --- .../main/scala/org/apache/spark/SparkContext.scala | 26 +++++++++++++++++++--- .../apache/spark/api/java/JavaSparkContext.scala | 14 ++++++++++++ core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- 3 files changed, 38 insertions(+), 4 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ad3337d94c..3ab27feffe 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -703,6 +703,26 @@ class SparkContext( } } + /** + * Support function for API backtraces. + */ + def setCallSite(site: String) { + setLocalProperty("externalCallSite", site) + } + + /** + * Support function for API backtraces. + */ + def clearCallSite() { + setLocalProperty("externalCallSite", null) + } + + private[spark] def getCallSite(): String = { + val callSite = getLocalProperty("externalCallSite") + if (callSite == null) return Utils.formatSparkCallSite + callSite + } + /** * Run a function on a given set of partitions in an RDD and pass the results to the given * handler function. This is the main entry point for all actions in Spark. The allowLocal @@ -715,7 +735,7 @@ class SparkContext( partitions: Seq[Int], allowLocal: Boolean, resultHandler: (Int, U) => Unit) { - val callSite = Utils.formatSparkCallSite + val callSite = getCallSite val cleanedFunc = clean(func) logInfo("Starting job: " + callSite) val start = System.nanoTime @@ -799,7 +819,7 @@ class SparkContext( func: (TaskContext, Iterator[T]) => U, evaluator: ApproximateEvaluator[U, R], timeout: Long): PartialResult[R] = { - val callSite = Utils.formatSparkCallSite + val callSite = getCallSite logInfo("Starting job: " + callSite) val start = System.nanoTime val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout, @@ -819,7 +839,7 @@ class SparkContext( resultFunc: => R): SimpleFutureAction[R] = { val cleanF = clean(processPartition) - val callSite = Utils.formatSparkCallSite + val callSite = getCallSite val waiter = dagScheduler.submitJob( rdd, (context: TaskContext, iter: Iterator[T]) => cleanF(iter), 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 acf328aa6a..3003e7afb5 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 @@ -405,6 +405,20 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] new JavaRDD(sc.checkpointFile(path)) } + + /** + * Pass-through to SparkContext.setCallSite. For API support only. + */ + def setCallSite(site: String) { + sc.setCallSite(site) + } + + /** + * Pass-through to SparkContext.setCallSite. For API support only. + */ + def clearCallSite() { + sc.clearCallSite() + } } object JavaSparkContext { diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index ea45566ad1..311ee4d510 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -938,7 +938,7 @@ abstract class RDD[T: ClassTag]( private var storageLevel: StorageLevel = StorageLevel.NONE /** Record user function generating this RDD. */ - @transient private[spark] val origin = Utils.formatSparkCallSite + @transient private[spark] val origin = sc.getCallSite private[spark] def elementClassTag: ClassTag[T] = classTag[T] -- cgit v1.2.3 From cd00225db9b90fc845fd1458831bdd9d014d1bb6 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 29 Dec 2013 14:03:39 -0500 Subject: Add SparkConf support in Python --- .../main/scala/org/apache/spark/SparkConf.scala | 14 ++- .../main/scala/org/apache/spark/SparkContext.scala | 2 +- python/pyspark/__init__.py | 5 +- python/pyspark/conf.py | 113 +++++++++++++++++++++ python/pyspark/context.py | 40 +++++--- python/pyspark/java_gateway.py | 1 + 6 files changed, 158 insertions(+), 17 deletions(-) create mode 100644 python/pyspark/conf.py (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index bd24cd19f2..670c8b4caa 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -74,13 +74,21 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { this } - /** Set an environment variable to be used when launching executors for this application. */ + /** + * Set an environment variable to be used when launching executors for this application. + * These variables are stored as properties of the form spark.executorEnv.VAR_NAME + * (for example spark.executorEnv.PATH) but this method makes them easier to set. + */ def setExecutorEnv(variable: String, value: String): SparkConf = { settings("spark.executorEnv." + variable) = value this } - /** Set multiple environment variables to be used when launching executors. */ + /** + * Set multiple environment variables to be used when launching executors. + * These variables are stored as properties of the form spark.executorEnv.VAR_NAME + * (for example spark.executorEnv.PATH) but this method makes them easier to set. + */ def setExecutorEnv(variables: Seq[(String, String)]): SparkConf = { for ((k, v) <- variables) { setExecutorEnv(k, v) @@ -135,7 +143,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { } /** Get all parameters as a list of pairs */ - def getAll: Seq[(String, String)] = settings.clone().toSeq + def getAll: Array[(String, String)] = settings.clone().toArray /** Get a parameter, falling back to a default if not set */ def getOrElse(k: String, defaultValue: String): String = { diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 0567f7f437..c109ff930c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -78,7 +78,7 @@ class SparkContext( * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters */ def this(master: String, appName: String, conf: SparkConf) = - this(conf.setMaster(master).setAppName(appName)) + this(conf.clone().setMaster(master).setAppName(appName)) /** * Alternative constructor that allows setting common Spark properties directly diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 1f35f6f939..f1b95acf09 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -28,6 +28,8 @@ Public classes: A broadcast variable that gets reused across tasks. - L{Accumulator} An "add-only" shared variable that tasks can only add values to. + - L{SparkConf} Access files shipped with jobs. - L{StorageLevel} @@ -38,10 +40,11 @@ import os sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j0.7.egg")) +from pyspark.conf import SparkConf from pyspark.context import SparkContext from pyspark.rdd import RDD from pyspark.files import SparkFiles from pyspark.storagelevel import StorageLevel -__all__ = ["SparkContext", "RDD", "SparkFiles", "StorageLevel"] +__all__ = ["SparkConf", "SparkContext", "RDD", "SparkFiles", "StorageLevel"] diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py new file mode 100644 index 0000000000..56e615c287 --- /dev/null +++ b/python/pyspark/conf.py @@ -0,0 +1,113 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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.conf import SparkConf +>>> from pyspark.context import SparkContext +>>> conf = SparkConf() +>>> conf.setMaster("local").setAppName("My app") + +>>> conf.get("spark.master") +u'local' +>>> conf.get("spark.appName") +u'My app' +>>> sc = SparkContext(conf=conf) +>>> sc.master +u'local' +>>> sc.appName +u'My app' +>>> sc.sparkHome == None +True + +>>> conf = SparkConf() +>>> conf.setSparkHome("/path") + +>>> conf.get("spark.home") +u'/path' +>>> conf.setExecutorEnv("VAR1", "value1") + +>>> conf.setExecutorEnv(pairs = [("VAR3", "value3"), ("VAR4", "value4")]) + +>>> conf.get("spark.executorEnv.VAR1") +u'value1' +>>> sorted(conf.getAll(), key=lambda p: p[0]) +[(u'spark.executorEnv.VAR1', u'value1'), (u'spark.executorEnv.VAR3', u'value3'), (u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')] +""" + + +class SparkConf(object): + def __init__(self, loadDefaults=False): + from pyspark.context import SparkContext + SparkContext._ensure_initialized() + self._jconf = SparkContext._jvm.SparkConf(loadDefaults) + + def set(self, key, value): + self._jconf.set(key, value) + return self + + def setMaster(self, value): + self._jconf.setMaster(value) + return self + + def setAppName(self, value): + self._jconf.setAppName(value) + return self + + def setSparkHome(self, value): + self._jconf.setSparkHome(value) + return self + + def setExecutorEnv(self, key=None, value=None, pairs=None): + if (key != None and pairs != None) or (key == None and pairs == None): + raise Exception("Either pass one key-value pair or a list of pairs") + elif key != None: + self._jconf.setExecutorEnv(key, value) + elif pairs != None: + for (k, v) in pairs: + self._jconf.setExecutorEnv(k, v) + return self + + def setAll(self, pairs): + for (k, v) in pairs: + self._jconf.set(k, v) + return self + + def get(self, key): + return self._jconf.get(key) + + def getOrElse(self, key, defaultValue): + return self._jconf.getOrElse(key, defaultValue) + + def getAll(self): + pairs = [] + for elem in self._jconf.getAll(): + pairs.append((elem._1(), elem._2())) + return pairs + + def contains(self, key): + return self._jconf.contains(key) + + +def _test(): + import doctest + (failure_count, test_count) = doctest.testmod(optionflags=doctest.ELLIPSIS) + if failure_count: + exit(-1) + + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/context.py b/python/pyspark/context.py index c0645b2847..97c1526afd 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -24,6 +24,7 @@ from tempfile import NamedTemporaryFile from pyspark import accumulators from pyspark.accumulators import Accumulator from pyspark.broadcast import Broadcast +from pyspark.conf import SparkConf from pyspark.files import SparkFiles from pyspark.java_gateway import launch_gateway from pyspark.serializers import PickleSerializer, BatchedSerializer, MUTF8Deserializer @@ -49,14 +50,14 @@ class SparkContext(object): _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, serializer=PickleSerializer()): + def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, + environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None): """ Create a new SparkContext. @param master: Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). - @param jobName: A name for your job, to display on the cluster web UI + @param appName: A name for your job, to display on the cluster web UI. @param sparkHome: Location where Spark is installed on cluster nodes. @param pyFiles: Collection of .zip or .py files to send to the cluster and add to PYTHONPATH. These can be paths on the local file @@ -67,6 +68,7 @@ class SparkContext(object): Java object. Set 1 to disable batching or -1 to use an unlimited batch size. @param serializer: The serializer for RDDs. + @param conf: A L{SparkConf} object setting Spark properties. >>> from pyspark.context import SparkContext @@ -79,10 +81,7 @@ class SparkContext(object): """ SparkContext._ensure_initialized(self) - self.master = master - self.jobName = jobName - self.sparkHome = sparkHome or None # None becomes null in Py4J - self.environment = environment or {} + self.conf = conf or SparkConf() self._batchSize = batchSize # -1 represents an unlimited batch size self._unbatched_serializer = serializer if batchSize == 1: @@ -91,10 +90,26 @@ class SparkContext(object): self.serializer = BatchedSerializer(self._unbatched_serializer, batchSize) + # Set parameters passed directly on our conf; these operations will be no-ops + # if the parameters were None + self.conf.setMaster(master) + self.conf.setAppName(appName) + self.conf.setSparkHome(sparkHome) + environment = environment or {} + for key, value in environment.iteritems(): + self.conf.setExecutorEnv(key, value) + + if not self.conf.contains("spark.master"): + raise Exception("A master URL must be set in your configuration") + if not self.conf.contains("spark.appName"): + raise Exception("An application name must be set in your configuration") + + self.master = self.conf.get("spark.master") + self.appName = self.conf.get("spark.appName") + self.sparkHome = self.conf.getOrElse("spark.home", None) + # Create the Java SparkContext through Py4J - empty_string_array = self._gateway.new_array(self._jvm.String, 0) - self._jsc = self._jvm.JavaSparkContext(master, jobName, sparkHome, - empty_string_array) + self._jsc = self._jvm.JavaSparkContext(self.conf._jconf) # Create a single Accumulator in Java that we'll send all our updates through; # they will be passed back to us through a TCP server @@ -105,6 +120,7 @@ class SparkContext(object): self._jvm.PythonAccumulatorParam(host, port)) self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python') + # Broadcast's __reduce__ method stores Broadcast instances here. # This allows other code to determine which Broadcast instances have # been pickled, so it can determine which Java broadcast objects to @@ -143,8 +159,8 @@ class SparkContext(object): @classmethod def setSystemProperty(cls, key, value): """ - Set a system property, such as spark.executor.memory. This must be - invoked before instantiating SparkContext. + Set a Java system property, such as spark.executor.memory. This must + must be invoked before instantiating SparkContext. """ SparkContext._ensure_initialized() SparkContext._jvm.java.lang.System.setProperty(key, value) diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index e615c1e9b6..128f078d12 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -60,6 +60,7 @@ def launch_gateway(): # Connect to the gateway gateway = JavaGateway(GatewayClient(port=port), auto_convert=False) # Import the classes used by PySpark + java_import(gateway.jvm, "org.apache.spark.SparkConf") java_import(gateway.jvm, "org.apache.spark.api.java.*") java_import(gateway.jvm, "org.apache.spark.api.python.*") java_import(gateway.jvm, "scala.Tuple2") -- cgit v1.2.3 From 0bd1900cbce5946999c38293852d8ccd4f838930 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 29 Dec 2013 15:38:46 -0500 Subject: Fix a few settings that were being read as system properties after merge --- .../org/apache/spark/scheduler/TaskSchedulerImpl.scala | 4 +++- .../org/apache/spark/scheduler/TaskSetManager.scala | 18 ++++++++++-------- .../spark/streaming/scheduler/JobScheduler.scala | 4 ++-- 3 files changed, 15 insertions(+), 11 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 56a038dc69..bffd990e16 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -47,10 +47,12 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode */ private[spark] class TaskSchedulerImpl( val sc: SparkContext, - val maxTaskFailures: Int = System.getProperty("spark.task.maxFailures", "4").toInt, + val maxTaskFailures: Int, isLocal: Boolean = false) extends TaskScheduler with Logging { + def this(sc: SparkContext) = this(sc, sc.conf.getOrElse("spark.task.maxFailures", "4").toInt) + val conf = sc.conf // How often to check for speculative tasks diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 9b95e418d8..d752e6f111 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -54,12 +54,14 @@ private[spark] class TaskSetManager( clock: Clock = SystemClock) extends Schedulable with Logging { + val conf = sched.sc.conf + // CPUs to request per task - val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toInt + val CPUS_PER_TASK = conf.getOrElse("spark.task.cpus", "1").toInt // Quantile of tasks at which to start speculation - val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble - val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble + val SPECULATION_QUANTILE = conf.getOrElse("spark.speculation.quantile", "0.75").toDouble + val SPECULATION_MULTIPLIER = conf.getOrElse("spark.speculation.multiplier", "1.5").toDouble // Serializer for closures and tasks. val env = SparkEnv.get @@ -118,7 +120,7 @@ private[spark] class TaskSetManager( // How frequently to reprint duplicate exceptions in full, in milliseconds val EXCEPTION_PRINT_INTERVAL = - System.getProperty("spark.logging.exceptionPrintInterval", "10000").toLong + conf.getOrElse("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 @@ -682,14 +684,14 @@ private[spark] class TaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = System.getProperty("spark.locality.wait", "3000") + val defaultWait = conf.getOrElse("spark.locality.wait", "3000") level match { case TaskLocality.PROCESS_LOCAL => - System.getProperty("spark.locality.wait.process", defaultWait).toLong + conf.getOrElse("spark.locality.wait.process", defaultWait).toLong case TaskLocality.NODE_LOCAL => - System.getProperty("spark.locality.wait.node", defaultWait).toLong + conf.getOrElse("spark.locality.wait.node", defaultWait).toLong case TaskLocality.RACK_LOCAL => - System.getProperty("spark.locality.wait.rack", defaultWait).toLong + conf.getOrElse("spark.locality.wait.rack", defaultWait).toLong case TaskLocality.ANY => 0L } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 9511ccfbed..7fd8d41c8c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -25,7 +25,7 @@ import org.apache.spark.streaming._ /** * This class schedules jobs to be run on Spark. It uses the JobGenerator to generate - * the jobs and runs them using a thread pool. Number of threads + * the jobs and runs them using a thread pool. Number of threads */ private[streaming] class JobScheduler(val ssc: StreamingContext) extends Logging { @@ -33,7 +33,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { initLogging() val jobSets = new ConcurrentHashMap[Time, JobSet] - val numConcurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt + val numConcurrentJobs = ssc.conf.getOrElse("spark.streaming.concurrentJobs", "1").toInt val executor = Executors.newFixedThreadPool(numConcurrentJobs) val generator = new JobGenerator(this) val listenerBus = new StreamingListenerBus() -- cgit v1.2.3 From 1ee7f5aee451eb5686f1c5ef251d1ebdddb0e847 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 29 Dec 2013 18:15:46 -0500 Subject: Fix a change that was lost during merge --- .../src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 4edc6a0d3f..897d47a9ad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -47,7 +47,8 @@ private[spark] class LocalActor( private val localExecutorId = "localhost" private val localExecutorHostname = "localhost" - val executor = new Executor(localExecutorId, localExecutorHostname, Seq.empty, isLocal = true) + val executor = new Executor( + localExecutorId, localExecutorHostname, scheduler.conf.getAll, isLocal = true) def receive = { case ReviveOffers => -- cgit v1.2.3 From 11540b798d622f3883cb40b20cc30ea7d894790a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 29 Dec 2013 18:44:06 -0500 Subject: Added tests for SparkConf and fixed a bug Typesafe Config caches system properties the first time it's invoked by default, ignoring later changes unless you do something special --- .../main/scala/org/apache/spark/SparkConf.scala | 1 + core/src/test/resources/spark.conf | 6 ++ .../scala/org/apache/spark/SparkConfSuite.scala | 110 +++++++++++++++++++++ 3 files changed, 117 insertions(+) create mode 100644 core/src/test/resources/spark.conf create mode 100644 core/src/test/scala/org/apache/spark/SparkConfSuite.scala (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 670c8b4caa..8cecaff5dd 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -26,6 +26,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { private val settings = new HashMap[String, String]() if (loadDefaults) { + ConfigFactory.invalidateCaches() val typesafeConfig = ConfigFactory.systemProperties() .withFallback(ConfigFactory.parseResources("spark.conf")) for (e <- typesafeConfig.entrySet().asScala if e.getKey.startsWith("spark.")) { diff --git a/core/src/test/resources/spark.conf b/core/src/test/resources/spark.conf new file mode 100644 index 0000000000..6c99bdcb7a --- /dev/null +++ b/core/src/test/resources/spark.conf @@ -0,0 +1,6 @@ +spark.test.intTestProperty = 1 + +spark.test { + stringTestProperty = "hi" + listTestProperty = ["a", "b"] +} diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala new file mode 100644 index 0000000000..aaf0b80fe9 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -0,0 +1,110 @@ +package org.apache.spark + +import org.scalatest.FunSuite + +class SparkConfSuite extends FunSuite with LocalSparkContext { + // This test uses the spark.conf in core/src/test/resources, which has a few test properties + test("loading from spark.conf") { + val conf = new SparkConf() + assert(conf.get("spark.test.intTestProperty") === "1") + assert(conf.get("spark.test.stringTestProperty") === "hi") + // NOTE: we don't use list properties yet, but when we do, we'll have to deal with this syntax + assert(conf.get("spark.test.listTestProperty") === "[a, b]") + } + + // This test uses the spark.conf in core/src/test/resources, which has a few test properties + test("system properties override spark.conf") { + try { + System.setProperty("spark.test.intTestProperty", "2") + val conf = new SparkConf() + assert(conf.get("spark.test.intTestProperty") === "2") + assert(conf.get("spark.test.stringTestProperty") === "hi") + } finally { + System.clearProperty("spark.test.intTestProperty") + } + } + + test("initializing without loading defaults") { + try { + System.setProperty("spark.test.intTestProperty", "2") + val conf = new SparkConf(false) + assert(!conf.contains("spark.test.intTestProperty")) + assert(!conf.contains("spark.test.stringTestProperty")) + } finally { + System.clearProperty("spark.test.intTestProperty") + } + } + + test("named set methods") { + val conf = new SparkConf(false) + + conf.setMaster("local[3]") + conf.setAppName("My app") + conf.setSparkHome("/path") + conf.setJars(Seq("a.jar", "b.jar")) + conf.setExecutorEnv("VAR1", "value1") + conf.setExecutorEnv(Seq(("VAR2", "value2"), ("VAR3", "value3"))) + + assert(conf.get("spark.master") === "local[3]") + assert(conf.get("spark.appName") === "My app") + assert(conf.get("spark.home") === "/path") + assert(conf.get("spark.jars") === "a.jar,b.jar") + assert(conf.get("spark.executorEnv.VAR1") === "value1") + assert(conf.get("spark.executorEnv.VAR2") === "value2") + assert(conf.get("spark.executorEnv.VAR3") === "value3") + + // Test the Java-friendly versions of these too + conf.setJars(Array("c.jar", "d.jar")) + conf.setExecutorEnv(Array(("VAR4", "value4"), ("VAR5", "value5"))) + assert(conf.get("spark.jars") === "c.jar,d.jar") + assert(conf.get("spark.executorEnv.VAR4") === "value4") + assert(conf.get("spark.executorEnv.VAR5") === "value5") + } + + test("basic get and set") { + val conf = new SparkConf(false) + assert(conf.getAll.toSet === Set()) + conf.set("k1", "v1") + conf.setAll(Seq(("k2", "v2"), ("k3", "v3"))) + assert(conf.getAll.toSet === Set(("k1", "v1"), ("k2", "v2"), ("k3", "v3"))) + conf.set("k1", "v4") + conf.setAll(Seq(("k2", "v5"), ("k3", "v6"))) + assert(conf.getAll.toSet === Set(("k1", "v4"), ("k2", "v5"), ("k3", "v6"))) + assert(conf.contains("k1"), "conf did not contain k1") + assert(!conf.contains("k4"), "conf contained k4") + assert(conf.get("k1") === "v4") + intercept[Exception] { conf.get("k4") } + assert(conf.getOrElse("k4", "not found") === "not found") + assert(conf.getOption("k1") === Some("v4")) + assert(conf.getOption("k4") === None) + } + + test("creating SparkContext without master and app name") { + val conf = new SparkConf(false) + intercept[SparkException] { sc = new SparkContext(conf) } + } + + test("creating SparkContext without master") { + val conf = new SparkConf(false).setAppName("My app") + intercept[SparkException] { sc = new SparkContext(conf) } + } + + test("creating SparkContext without app name") { + val conf = new SparkConf(false).setMaster("local") + intercept[SparkException] { sc = new SparkContext(conf) } + } + + test("creating SparkContext with both master and app name") { + val conf = new SparkConf(false).setMaster("local").setAppName("My app") + sc = new SparkContext(conf) + assert(sc.master === "local") + assert(sc.appName === "My app") + } + + test("SparkContext property overriding") { + val conf = new SparkConf(false).setMaster("local").setAppName("My app") + sc = new SparkContext("local[2]", "My other app", conf) + assert(sc.master === "local[2]") + assert(sc.appName === "My other app") + } +} -- cgit v1.2.3 From 994f080f8ae3372366e6004600ba791c8a372ff0 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 29 Dec 2013 22:19:33 -0500 Subject: Properly show Spark properties on web UI, and change app name property --- core/src/main/scala/org/apache/spark/SparkConf.scala | 2 +- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 ++-- .../main/scala/org/apache/spark/ui/env/EnvironmentUI.scala | 13 ++++++++----- core/src/test/scala/org/apache/spark/SparkConfSuite.scala | 2 +- python/pyspark/conf.py | 2 +- python/pyspark/context.py | 4 ++-- 6 files changed, 15 insertions(+), 12 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 8cecaff5dd..ae52de409e 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -54,7 +54,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { /** Set a name for your application. Shown in the Spark web UI. */ def setAppName(name: String): SparkConf = { if (name != null) { - settings("spark.appName") = name + settings("spark.app.name") = name } this } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 6f54fa7a5a..810ed1860b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -112,7 +112,7 @@ class SparkContext( if (!conf.contains("spark.master")) { throw new SparkException("A master URL must be set in your configuration") } - if (!conf.contains("spark.appName")) { + if (!conf.contains("spark.app.name")) { throw new SparkException("An application must be set in your configuration") } @@ -127,7 +127,7 @@ class SparkContext( } val master = conf.get("spark.master") - val appName = conf.get("spark.appName") + val appName = conf.get("spark.app.name") val isLocal = (master == "local" || master.startsWith("local[")) diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 91fa00a66c..6b4602f928 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -48,12 +48,15 @@ private[spark] class EnvironmentUI(sc: SparkContext) { def jvmTable = UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation, fixedWidth = true) - val properties = System.getProperties.iterator.toSeq - val classPathProperty = properties.find { case (k, v) => - k.contains("java.class.path") + val sparkProperties = sc.conf.getAll.sorted + + val systemProperties = System.getProperties.iterator.toSeq + val classPathProperty = systemProperties.find { case (k, v) => + k == "java.class.path" }.getOrElse(("", "")) - val sparkProperties = properties.filter(_._1.startsWith("spark")).sorted - val otherProperties = properties.diff(sparkProperties :+ classPathProperty).sorted + val otherProperties = systemProperties.filter { case (k, v) => + k != "java.class.path" && !k.startsWith("spark.") + }.sorted val propertyHeaders = Seq("Name", "Value") def propertyRow(kv: (String, String)) =
    diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index aaf0b80fe9..77c7b829b3 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -46,7 +46,7 @@ class SparkConfSuite extends FunSuite with LocalSparkContext { conf.setExecutorEnv(Seq(("VAR2", "value2"), ("VAR3", "value3"))) assert(conf.get("spark.master") === "local[3]") - assert(conf.get("spark.appName") === "My app") + assert(conf.get("spark.app.name") === "My app") assert(conf.get("spark.home") === "/path") assert(conf.get("spark.jars") === "a.jar,b.jar") assert(conf.get("spark.executorEnv.VAR1") === "value1") diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index cf98b0e071..c07dd88307 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -23,7 +23,7 @@ >>> conf.get("spark.master") u'local' ->>> conf.get("spark.appName") +>>> conf.get("spark.app.name") u'My app' >>> sc = SparkContext(conf=conf) >>> sc.master diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 8b028027eb..12ac0299e2 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -104,13 +104,13 @@ class SparkContext(object): # Check that we have at least the required parameters if not self.conf.contains("spark.master"): raise Exception("A master URL must be set in your configuration") - if not self.conf.contains("spark.appName"): + if not self.conf.contains("spark.app.name"): raise Exception("An application name must be set in your configuration") # Read back our properties from the conf in case we loaded some of them from # the classpath or an external config file self.master = self.conf.get("spark.master") - self.appName = self.conf.get("spark.appName") + self.appName = self.conf.get("spark.app.name") self.sparkHome = self.conf.getOrElse("spark.home", None) for (k, v) in self.conf.getAll(): if k.startswith("spark.executorEnv."): -- cgit v1.2.3 From c2c1af39f593cd00d29368efe2dbb8c0444f624d Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Sun, 29 Dec 2013 21:10:08 -0800 Subject: Updated code style according to Patrick's comments --- core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) (limited to 'core/src') 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 3becb4f068..627995c826 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -119,8 +119,7 @@ object StatsReportListener extends Logging { val probabilities = percentiles.map{_ / 100.0} val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" - def extractDoubleDistribution( - stage:SparkListenerStageCompleted, + def extractDoubleDistribution(stage: SparkListenerStageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Double]) : Option[Distribution] = { Distribution(stage.stage.taskInfos.flatMap { @@ -128,8 +127,7 @@ object StatsReportListener extends Logging { } //is there some way to setup the types that I can get rid of this completely? - def extractLongDistribution( - stage:SparkListenerStageCompleted, + def extractLongDistribution(stage: SparkListenerStageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Long]) : Option[Distribution] = { extractDoubleDistribution(stage, (info, metric) => getMetric(info,metric).map{_.toDouble}) -- cgit v1.2.3 From cffe1c1d5c0abfbb463fa91e8b340a9c535532fe Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 29 Dec 2013 20:29:29 -0800 Subject: SPARK-1008: Logging improvments 1. Adds a default log4j file that gets loaded if users haven't specified a log4j file. 2. Isolates use of the tools assembly jar. I found this produced SLF4J warnings after building with SBT (and I've seen similar warnings on the mailing list). --- .../org/apache/spark/default-log4j.properties | 8 ++++++++ core/src/main/scala/org/apache/spark/Logging.scala | 18 +++++++++++++++--- .../scala/org/apache/spark/repl/SparkILoop.scala | 2 ++ spark-class | 22 ++++++++++++---------- 4 files changed, 37 insertions(+), 13 deletions(-) create mode 100644 core/src/main/resources/org/apache/spark/default-log4j.properties (limited to 'core/src') diff --git a/core/src/main/resources/org/apache/spark/default-log4j.properties b/core/src/main/resources/org/apache/spark/default-log4j.properties new file mode 100644 index 0000000000..d72dbadc39 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/default-log4j.properties @@ -0,0 +1,8 @@ +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %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/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 6a973ea495..a8892737fa 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -17,8 +17,8 @@ package org.apache.spark -import org.slf4j.Logger -import org.slf4j.LoggerFactory +import org.apache.log4j.{LogManager, PropertyConfigurator} +import org.slf4j.{Logger, LoggerFactory} /** * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows @@ -91,5 +91,17 @@ trait Logging { // Method for ensuring that logging is initialized, to avoid having multiple // threads do it concurrently (as SLF4J initialization is not thread safe). - protected def initLogging() { log } + protected def initLogging() { + // If Log4j doesn't seem initialized, load a default properties file + def log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements + if (!log4jInitialized) { + val defaultLogProps = "org/apache/spark/default-log4j.properties" + val classLoader = this.getClass.getClassLoader + Option(classLoader.getResource(defaultLogProps)) match { + case Some(url) => PropertyConfigurator.configure(url) + case None => System.err.println(s"Spark was unable to load $defaultLogProps") + } + } + log + } } diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 523fd1222d..b2a1815459 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -60,6 +60,8 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out, None) def this() = this(None, new JPrintWriter(Console.out, true), None) + initLogging() + var in: InteractiveReader = _ // the input stream from which commands come var settings: Settings = _ var intp: SparkIMain = _ diff --git a/spark-class b/spark-class index 802e4aa104..1c4323ee53 100755 --- a/spark-class +++ b/spark-class @@ -115,16 +115,18 @@ if [ ! -f "$FWDIR/RELEASE" ]; then fi fi -TOOLS_DIR="$FWDIR"/tools -SPARK_TOOLS_JAR="" -if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then - # Use the JAR from the SBT build - export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar` -fi -if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then - # Use the JAR from the Maven build - # TODO: this also needs to become an assembly! - export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar` +if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then + TOOLS_DIR="$FWDIR"/tools + SPARK_TOOLS_JAR="" + if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then + # Use the JAR from the SBT build + export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar` + fi + if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then + # Use the JAR from the Maven build + # TODO: this also needs to become an assembly! + export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar` + fi fi # Compute classpath using external script -- cgit v1.2.3 From 1cbef081e3d1dcc647b49a2f2b5f13ceaa0f611d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 30 Dec 2013 12:46:09 -0800 Subject: Response to Shivaram's review --- core/src/main/scala/org/apache/spark/Logging.scala | 2 +- spark-class | 31 ++++++++++++---------- 2 files changed, 18 insertions(+), 15 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index a8892737fa..b97697d587 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -93,7 +93,7 @@ trait Logging { // threads do it concurrently (as SLF4J initialization is not thread safe). protected def initLogging() { // If Log4j doesn't seem initialized, load a default properties file - def log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements + val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements if (!log4jInitialized) { val defaultLogProps = "org/apache/spark/default-log4j.properties" val classLoader = this.getClass.getClassLoader diff --git a/spark-class b/spark-class index 1c4323ee53..1858ea6247 100755 --- a/spark-class +++ b/spark-class @@ -115,27 +115,30 @@ if [ ! -f "$FWDIR/RELEASE" ]; then fi fi -if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then - TOOLS_DIR="$FWDIR"/tools - SPARK_TOOLS_JAR="" - if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then - # Use the JAR from the SBT build - export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar` - fi - if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then - # Use the JAR from the Maven build - # TODO: this also needs to become an assembly! - export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar` - fi +TOOLS_DIR="$FWDIR"/tools +SPARK_TOOLS_JAR="" +if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then + # Use the JAR from the SBT build + export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar` +fi +if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then + # Use the JAR from the Maven build + # TODO: this also needs to become an assembly! + export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar` fi # Compute classpath using external script CLASSPATH=`$FWDIR/bin/compute-classpath.sh` -CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" + +if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then + CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" +fi if $cygwin; then CLASSPATH=`cygpath -wp $CLASSPATH` - export SPARK_TOOLS_JAR=`cygpath -w $SPARK_TOOLS_JAR` + if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then + export SPARK_TOOLS_JAR=`cygpath -w $SPARK_TOOLS_JAR` + fi fi export CLASSPATH -- cgit v1.2.3 From 0fa5809768cf60ec62b4277f04e23a44dc1582e2 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 30 Dec 2013 22:17:28 -0500 Subject: Updated docs for SparkConf and handled review comments --- .../main/scala/org/apache/spark/Partitioner.scala | 2 +- .../main/scala/org/apache/spark/SparkConf.scala | 31 ++++++---- .../main/scala/org/apache/spark/SparkContext.scala | 18 ++++-- .../src/main/scala/org/apache/spark/SparkEnv.scala | 13 ++-- .../apache/spark/api/java/JavaSparkContext.scala | 6 ++ .../apache/spark/deploy/FaultToleranceTest.scala | 4 +- .../scala/org/apache/spark/executor/Executor.scala | 2 +- .../main/scala/org/apache/spark/util/Utils.scala | 10 +-- core/src/test/resources/spark.conf | 2 + docs/_config.yml | 2 +- docs/configuration.md | 71 +++++++++++++++++----- docs/css/bootstrap.min.css | 2 +- docs/job-scheduling.md | 21 ++++--- docs/monitoring.md | 3 +- docs/python-programming-guide.md | 15 ++--- docs/quick-start.md | 52 +++++++++++----- docs/running-on-mesos.md | 19 +++--- docs/scala-programming-guide.md | 4 +- docs/spark-standalone.md | 15 +++-- docs/streaming-programming-guide.md | 4 +- docs/tuning.md | 21 ++++--- python/pyspark/conf.py | 24 ++++++-- python/pyspark/context.py | 24 ++++---- 23 files changed, 241 insertions(+), 124 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 7cb545a6be..31b0773bfe 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -52,7 +52,7 @@ object Partitioner { for (r <- bySize if r.partitioner != None) { return r.partitioner.get } - if (rdd.context.conf.getOrElse("spark.default.parallelism", null) != null) { + if (rdd.context.conf.contains("spark.default.parallelism")) { return new HashPartitioner(rdd.context.defaultParallelism) } else { return new HashPartitioner(bySize.head.partitions.size) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index ae52de409e..96239cf4be 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -16,6 +16,12 @@ import com.typesafe.config.ConfigFactory * For unit tests, you can also call `new SparkConf(false)` to skip loading external settings and * get the same configuration no matter what is on the classpath. * + * All setter methods in this class support chaining. For example, you can write + * `new SparkConf().setMaster("local").setAppName("My app")`. + * + * Note that once a SparkConf object is passed to Spark, it is cloned and can no longer be modified + * by the user. Spark does not support modifying the configuration at runtime. + * * @param loadDefaults whether to load values from the system properties and classpath */ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { @@ -69,10 +75,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { /** Set JAR files to distribute to the cluster. (Java-friendly version.) */ def setJars(jars: Array[String]): SparkConf = { - if (!jars.isEmpty) { - settings("spark.jars") = jars.mkString(",") - } - this + setJars(jars.toSeq) } /** @@ -102,15 +105,11 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { * (Java-friendly version.) */ def setExecutorEnv(variables: Array[(String, String)]): SparkConf = { - for ((k, v) <- variables) { - setExecutorEnv(k, v) - } - this + setExecutorEnv(variables.toSeq) } /** - * Set the location where Spark is installed on worker nodes. This is only needed on Mesos if - * you are not using `spark.executor.uri` to disseminate the Spark binary distribution. + * Set the location where Spark is installed on worker nodes. */ def setSparkHome(home: String): SparkConf = { if (home != null) { @@ -154,8 +153,8 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { /** Get all executor environment variables set on this SparkConf */ def getExecutorEnv: Seq[(String, String)] = { val prefix = "spark.executorEnv." - getAll.filter(pair => pair._1.startsWith(prefix)) - .map(pair => (pair._1.substring(prefix.length), pair._2)) + getAll.filter{case (k, v) => k.startsWith(prefix)} + .map{case (k, v) => (k.substring(prefix.length), v)} } /** Does the configuration contain a given parameter? */ @@ -165,4 +164,12 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { override def clone: SparkConf = { new SparkConf(false).setAll(settings) } + + /** + * Return a string listing all keys and values, one per line. This is useful to print the + * configuration out for debugging. + */ + def toDebugString: String = { + settings.toArray.sorted.map{case (k, v) => k + "=" + v}.mkString("\n") + } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 810ed1860b..8134ce7eb3 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -55,14 +55,14 @@ import org.apache.spark.util._ * 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. * - * @param conf_ a Spark Config object describing the application configuration. Any settings in + * @param config a Spark Config object describing the application configuration. Any settings in * this config overrides the default configs as well as system properties. * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Can * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] * from a list of input files or InputFormats for the application. */ class SparkContext( - conf_ : SparkConf, + config: SparkConf, // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, etc) // too. This is typically generated from InputFormatInfo.computePreferredLocations. It contains // a map from hostname to a list of input format splits on the host. @@ -107,7 +107,13 @@ class SparkContext( preferredNodeLocationData) } - val conf = conf_.clone() + private[spark] val conf = config.clone() + + /** + * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be + * changed at runtime. + */ + def getConf: SparkConf = conf.clone() if (!conf.contains("spark.master")) { throw new SparkException("A master URL must be set in your configuration") @@ -135,11 +141,11 @@ class SparkContext( initLogging() // Create the Spark execution environment (cache, map output tracker, etc) - private[spark] val env = SparkEnv.createFromSystemProperties( + private[spark] val env = SparkEnv.create( + conf, "", conf.get("spark.driver.host"), conf.get("spark.driver.port").toInt, - conf, isDriver = true, isLocal = isLocal) SparkEnv.set(env) @@ -730,7 +736,7 @@ class SparkContext( * (in that order of preference). If neither of these is set, return None. */ private[spark] def getSparkHome(): Option[String] = { - if (conf.getOrElse("spark.home", null) != null) { + if (conf.contains("spark.home")) { Some(conf.get("spark.home")) } else if (System.getenv("SPARK_HOME") != null) { Some(System.getenv("SPARK_HOME")) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 34fad3e763..d06af8e667 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -40,7 +40,7 @@ import com.google.common.collect.MapMaker * objects needs to have the right SparkEnv set. You can get the current environment with * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set. */ -class SparkEnv ( +class SparkEnv private[spark] ( val executorId: String, val actorSystem: ActorSystem, val serializerManager: SerializerManager, @@ -63,7 +63,7 @@ class SparkEnv ( // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats). private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]() - def stop() { + private[spark] def stop() { pythonWorkers.foreach { case(key, worker) => worker.stop() } httpFileServer.stop() mapOutputTracker.stop() @@ -79,6 +79,7 @@ class SparkEnv ( //actorSystem.awaitTermination() } + private[spark] def createPythonWorker(pythonExec: String, envVars: Map[String, String]): java.net.Socket = { synchronized { val key = (pythonExec, envVars) @@ -111,11 +112,11 @@ object SparkEnv extends Logging { env.get() } - def createFromSystemProperties( + private[spark] def create( + conf: SparkConf, executorId: String, hostname: String, port: Int, - conf: SparkConf, isDriver: Boolean, isLocal: Boolean): SparkEnv = { @@ -129,7 +130,7 @@ object SparkEnv extends Logging { } // set only if unset until now. - if (conf.getOrElse("spark.hostPort", null) == null) { + if (!conf.contains("spark.hostPort")) { if (!isDriver){ // unexpected Utils.logErrorWithStack("Unexpected NOT to have spark.hostPort set") @@ -216,7 +217,7 @@ object SparkEnv extends Logging { } // Warn about deprecated spark.cache.class property - if (conf.getOrElse("spark.cache.class", null) != null) { + if (conf.contains("spark.cache.class")) { logWarning("The spark.cache.class property is no longer being used! Specify storage " + "levels using the RDD.persist() method instead.") } 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 e03cf9d13a..d6aeed7661 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 @@ -418,6 +418,12 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] new JavaRDD(sc.checkpointFile(path)) } + + /** + * Return a copy of this JavaSparkContext's configuration. The configuration ''cannot'' be + * changed at runtime. + */ + def getConf: SparkConf = sc.getConf } object JavaSparkContext { diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 0aa8852649..4dfb19ed8a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -190,7 +190,7 @@ private[spark] object FaultToleranceTest extends App with Logging { /** Creates a SparkContext, which constructs a Client to interact with our cluster. */ def createClient() = { if (sc != null) { sc.stop() } - // Counter-hack: Because of a hack in SparkEnv#createFromSystemProperties() that changes this + // Counter-hack: Because of a hack in SparkEnv#create() that changes this // property, we need to reset it. System.setProperty("spark.driver.port", "0") sc = new SparkContext(getMasterUrls(masters), "fault-tolerance", containerSparkHome) @@ -417,4 +417,4 @@ private[spark] object Docker extends Logging { "docker ps -l -q".!(ProcessLogger(line => id = line)) new DockerId(id) } -} \ No newline at end of file +} 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 a6eabc462b..2400154648 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -109,7 +109,7 @@ private[spark] class Executor( // Initialize Spark environment (using system properties read above) private val env = { if (!isLocal) { - val _env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, conf, + val _env = SparkEnv.create(conf, executorId, slaveHostname, 0, isDriver = false, isLocal = false) SparkEnv.set(_env) _env.metricsSystem.registerSource(executorSource) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index b6b89cc7bb..ca3320b22b 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -397,12 +397,11 @@ private[spark] object Utils extends Logging { } def localHostPort(conf: SparkConf): String = { - val retval = conf.getOrElse("spark.hostPort", null) + val retval = conf.getOrElse("spark.hostPort", null) if (retval == null) { logErrorWithStack("spark.hostPort not set but invoking localHostPort") return localHostName() } - retval } @@ -414,9 +413,12 @@ private[spark] object Utils extends Logging { 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) } } + try { + throw new Exception + } catch { + case ex: Exception => logError(msg, ex) + } } // Typically, this will be of order of number of nodes in cluster diff --git a/core/src/test/resources/spark.conf b/core/src/test/resources/spark.conf index 6c99bdcb7a..aa4e751235 100644 --- a/core/src/test/resources/spark.conf +++ b/core/src/test/resources/spark.conf @@ -1,3 +1,5 @@ +# A simple spark.conf file used only in our unit tests + spark.test.intTestProperty = 1 spark.test { diff --git a/docs/_config.yml b/docs/_config.yml index 02067f9750..11d18f0ac2 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -4,7 +4,7 @@ markdown: kramdown # These allow the documentation to be updated with nerw releases # of Spark, Scala, and Mesos. SPARK_VERSION: 0.9.0-incubating-SNAPSHOT -SPARK_VERSION_SHORT: 0.9.0-SNAPSHOT +SPARK_VERSION_SHORT: 0.9.0 SCALA_VERSION: 2.10 MESOS_VERSION: 0.13.0 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net diff --git a/docs/configuration.md b/docs/configuration.md index 677d182e50..567aba07f0 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -3,26 +3,37 @@ layout: global title: Spark Configuration --- -Spark provides three main locations to configure the system: +Spark provides three locations to configure the system: -* [Java system properties](#system-properties), which control internal configuration parameters and can be set - either programmatically (by calling `System.setProperty` *before* creating a `SparkContext`) or through - JVM arguments. -* [Environment variables](#environment-variables) for configuring per-machine settings such as the IP address, - which can be set in the `conf/spark-env.sh` script. -* [Logging configuration](#configuring-logging), which is done through `log4j.properties`. +* [Spark properties](#spark-properties) control most application parameters and can be set by passing + a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java + system properties. +* [Environment variables](#environment-variables) can be used to set per-machine settings, such as + the IP address, through the `conf/spark-env.sh` script on each node. +* [Logging](#configuring-logging) can be configured through `log4j.properties`. -# System Properties +# Spark Properties -To set a system property for configuring Spark, you need to either pass it with a -D flag to the JVM (for example `java -Dspark.cores.max=5 MyProgram`) or call `System.setProperty` in your code *before* creating your Spark context, as follows: +Spark properties control most application settings and are configured separately for each application. +The preferred way to set them is by passing a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) +class to your SparkContext constructor. +Alternatively, Spark will also load them from Java system properties (for compatibility with old versions +of Spark) and from a [`spark.conf` file](#configuration-files) on your classpath. + +SparkConf lets you configure most of the common properties to initialize a cluster (e.g., master URL and +application name), as well as arbitrary key-value pairs through the `set()` method. For example, we could +initialize an application as follows: {% highlight scala %} -System.setProperty("spark.cores.max", "5") -val sc = new SparkContext(...) +val conf = new SparkConf() + .setMaster("local") + .setAppName("My application") + .set("spark.executor.memory", "1g") +val sc = new SparkContext(conf) {% endhighlight %} -Most of the configurable system properties control internal settings that have reasonable default values. However, +Most of the properties control internal settings that have reasonable default values. However, there are at least five properties that you will commonly want to control:
    Executor IDAddress Task Time Total Tasks Failed Tasks
    {k}{executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} {parent.formatDuration(v.taskTime)} {v.failedTasks + v.succeededTasks} {v.failedTasks}
    {kv._1}{kv._2}
    @@ -385,11 +396,40 @@ Apart from these, the following properties are also available, and may be useful
    +## Viewing Spark Properties + +The application web UI at `http://:4040` lists Spark properties in the "Environment" tab. +This is a useful place to check to make sure that your properties have been set correctly. + +## Configuration Files + +You can also configure Spark properties through a `spark.conf` file on your Java classpath. +Because these properties are usually application-specific, we recommend putting this fine *only* on your +application's classpath, and not in a global Spark classpath. + +The `spark.conf` file uses Typesafe Config's [HOCON format](https://github.com/typesafehub/config#json-superset), +which is a superset of Java properties files and JSON. For example, the following is a simple config file: + +{% highlight awk %} +# Comments are allowed +spark.executor.memory = 512m +spark.serializer = org.apache.spark.serializer.KryoSerializer +{% endhighlight %} + +The format also allows hierarchical nesting, as follows: + +{% highlight awk %} +spark.akka { + threads = 8 + timeout = 200 +} +{% endhighlight %} + # Environment Variables -Certain Spark settings can also be configured through environment variables, which are read from the `conf/spark-env.sh` +Certain Spark settings can be configured through environment variables, which are read from the `conf/spark-env.sh` 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 +as library search paths. While Spark properties can also be set there through `SPARK_JAVA_OPTS`, for per-application settings, we recommend setting these properties within the application instead of in `spark-env.sh` so that different applications can use different settings. @@ -406,7 +446,8 @@ The following variables can be set in `spark-env.sh`: Note that applications can also add dependencies for themselves through `SparkContext.addJar` -- we recommend doing that when possible. * `SPARK_JAVA_OPTS`, to add JVM options. This includes Java options like garbage collector settings and any system - properties that you'd like to pass with `-D` (e.g., `-Dspark.local.dir=/disk1,/disk2`). + properties that you'd like to pass with `-D`. One use case is to set some Spark properties differently on this + machine, e.g., `-Dspark.local.dir=/disk1,/disk2`. * Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores to use on each machine and maximum memory. diff --git a/docs/css/bootstrap.min.css b/docs/css/bootstrap.min.css index ccb529eaba..119bb5dba7 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,#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} + */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,#feecda);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#feecda));background-image:-webkit-linear-gradient(top,#fff,#feecda);background-image:-o-linear-gradient(top,#fff,#feecda);background-image:linear-gradient(to bottom,#fff,#feecda);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='#fffeecda',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/job-scheduling.md b/docs/job-scheduling.md index dbcb9ae343..5951155fe3 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -32,12 +32,12 @@ Resource allocation can be configured as follows, based on the cluster type: * **Standalone mode:** By default, applications submitted to the standalone mode cluster will run in FIFO (first-in-first-out) order, and each application will try to use all available nodes. You can limit - the number of nodes an application uses by setting the `spark.cores.max` system property in it. This + the number of nodes an application uses by setting the `spark.cores.max` configuration property in it. This will allow multiple users/applications to run concurrently. For example, you might launch a long-running server that uses 10 cores, and allow users to launch shells that use 20 cores each. Finally, in addition to controlling cores, each application's `spark.executor.memory` setting controls its memory use. -* **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` system property to `true`, +* **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` configuration property to `true`, and optionally set `spark.cores.max` to limit each application's resource share as in the standalone mode. You should also set `spark.executor.memory` to control the executor memory. * **YARN:** The `--num-workers` option to the Spark YARN client controls how many workers it will allocate @@ -78,11 +78,13 @@ of cluster resources. This means that short jobs submitted while a long job is r resources right away and still get good response times, without waiting for the long job to finish. This mode is best for multi-user settings. -To enable the fair scheduler, simply set the `spark.scheduler.mode` to `FAIR` before creating +To enable the fair scheduler, simply set the `spark.scheduler.mode` property to `FAIR` when configuring a SparkContext: {% highlight scala %} -System.setProperty("spark.scheduler.mode", "FAIR") +val conf = new SparkConf().setMaster(...).setAppName(...) +conf.set("spark.scheduler.mode", "FAIR") +val sc = new SparkContext(conf) {% endhighlight %} ## Fair Scheduler Pools @@ -98,8 +100,8 @@ adding the `spark.scheduler.pool` "local property" to the SparkContext in the th This is done as follows: {% highlight scala %} -// Assuming context is your SparkContext variable -context.setLocalProperty("spark.scheduler.pool", "pool1") +// Assuming sc is your SparkContext variable +sc.setLocalProperty("spark.scheduler.pool", "pool1") {% endhighlight %} After setting this local property, _all_ jobs submitted within this thread (by calls in this thread @@ -108,7 +110,7 @@ it easy to have a thread run multiple jobs on behalf of the same user. If you'd pool that a thread is associated with, simply call: {% highlight scala %} -context.setLocalProperty("spark.scheduler.pool", null) +sc.setLocalProperty("spark.scheduler.pool", null) {% endhighlight %} ## Default Behavior of Pools @@ -138,10 +140,11 @@ properties: of the cluster. By default, each pool's `minShare` is 0. The pool properties can be set by creating an XML file, similar to `conf/fairscheduler.xml.template`, -and setting the `spark.scheduler.allocation.file` property: +and setting a `spark.scheduler.allocation.file` property in your +[SparkConf](configuration.html#spark-properties). {% highlight scala %} -System.setProperty("spark.scheduler.allocation.file", "/path/to/file") +conf.set("spark.scheduler.allocation.file", "/path/to/file") {% endhighlight %} The format of the XML file is simply a `` element for each pool, with different elements diff --git a/docs/monitoring.md b/docs/monitoring.md index 5ed0474477..0d5eb7065e 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -32,7 +32,8 @@ Spark has a configurable metrics system based on the This allows users to report Spark metrics to a variety of sinks including HTTP, JMX, and CSV files. The metrics system is configured via a configuration file that Spark expects to be present at `$SPARK_HOME/conf/metrics.conf`. A custom file location can be specified via the -`spark.metrics.conf` Java system property. Spark's metrics are decoupled into different +`spark.metrics.conf` [configuration property](configuration.html#spark-properties). +Spark's metrics are decoupled into different _instances_ corresponding to Spark components. Within each instance, you can configure a set of sinks to which metrics are reported. The following instances are currently supported: diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 55e39b1de1..96f93e24fe 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -131,15 +131,16 @@ sc = SparkContext("local", "App 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. -You can set [system properties](configuration.html#system-properties) -using `SparkContext.setSystemProperty()` class method *before* -instantiating SparkContext. For example, to set the amount of memory -per executor process: +You can set [configuration properties](configuration.html#spark-properties) by passing a +[SparkConf](api/pyspark/pyspark.conf.SparkConf-class.html) object to SparkContext: {% highlight python %} -from pyspark import SparkContext -SparkContext.setSystemProperty('spark.executor.memory', '2g') -sc = SparkContext("local", "App Name") +from pyspark import SparkConf, SparkContext +conf = (SparkConf() + .setMaster("local") + .setAppName("My app") + .set("spark.executor.memory", "1g")) +sc = SparkContext(conf = conf) {% endhighlight %} # API Docs diff --git a/docs/quick-start.md b/docs/quick-start.md index 8f782db5b8..bdf127d404 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -124,7 +124,7 @@ object SimpleApp { } {% endhighlight %} -This program 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 proogram. 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 application, the directory where Spark is installed, and a name for the jar file containing the application's code. 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 program just 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 proogram. 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 application, the directory where Spark is installed, and a name for the jar file containing the application's code. 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 a repository that Spark depends on: @@ -193,7 +193,7 @@ public class SimpleApp { } {% endhighlight %} -This program simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail. +This program just counts the number of lines containing 'a' and the number containing 'b' in a text file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail. To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version. @@ -224,11 +224,11 @@ To build the program, we also write a Maven `pom.xml` file that lists Spark as a 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: {% highlight xml %} - - org.apache.hadoop - hadoop-client - ... - + + org.apache.hadoop + hadoop-client + ... + {% endhighlight %} We lay out these files according to the canonical Maven directory structure: @@ -270,7 +270,7 @@ print "Lines with a: %i, lines with b: %i" % (numAs, numBs) {% endhighlight %} -This program simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. +This program just counts the number of lines containing 'a' and the number containing 'b' in a text file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala and Java examples, we use a SparkContext to create RDDs. We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference. @@ -301,14 +301,38 @@ assembly jar (or "uber" jar) containing your code and its dependencies. Both 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. +add it to the SparkContext as shown here. It is also possible to add +your dependent jars one-by-one using the `addJar` method of `SparkContext`. + +For Python, you can use the `pyFiles` argument of SparkContext +or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed. ### Setting Configuration Options -Spark includes several configuration options which influence the behavior -of your application. 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. +Spark includes several [configuration options](configuration.html#spark-properties) +that influence the behavior of your application. +These should be set by building a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) +object and passing it to the SparkContext constructor. +For example, in Java and Scala, you can do: + +{% highlight scala %} +import org.apache.spark.{SparkConf, SparkContext} +val conf = new SparkConf() + .setMaster("local") + .setAppName("My application") + .set("spark.executor.memory", "1g") +val sc = new SparkContext(conf) +{% endhighlight %} + +Or in Python: + +{% highlight scala %} +from pyspark import SparkConf, SparkContext +conf = SparkConf() +conf.setMaster("local") +conf.setAppName("My application") +conf.set("spark.executor.memory", "1g")) +sc = SparkContext(conf = conf) +{% endhighlight %} ### Accessing Hadoop Filesystems diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 322ff585f1..68259f0cb8 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -15,15 +15,16 @@ Spark can run on clusters managed by [Apache Mesos](http://mesos.apache.org/). F * `export MESOS_NATIVE_LIBRARY=`. This path is usually `/lib/libmesos.so` (where the prefix is `/usr/local` by default, see above). Also, on Mac OS X, the library is called `libmesos.dylib` instead of `libmesos.so`. * `export SPARK_EXECUTOR_URI=`. * `export MASTER=mesos://HOST:PORT` where HOST:PORT is the host and port (default: 5050) of your Mesos master (or `zk://...` if using Mesos with ZooKeeper). -8. To run a Spark application against the cluster, when you create your `SparkContext`, pass the string `mesos://HOST:PORT` as the first parameter. In addition, you'll need to set the `spark.executor.uri` property. For example: +8. To run a Spark application against the cluster, when you create your `SparkContext`, pass the string `mesos://HOST:PORT` as the master URL. In addition, you'll need to set the `spark.executor.uri` property. For example: {% highlight scala %} -System.setProperty("spark.executor.uri", "") -val sc = new SparkContext("mesos://HOST:5050", "App Name", ...) +val conf = new SparkConf() + .setMaster("mesos://HOST:5050") + .setAppName("My app") + .set("spark.executor.uri", "") +val sc = new SparkContext(conf) {% endhighlight %} -If you want to run Spark on Amazon EC2, you can use the Spark [EC2 launch scripts](ec2-scripts.html), which provide an easy way to launch a cluster with Mesos, Spark, and HDFS pre-configured. This will get you a cluster in about five minutes without any configuration on your part. - # Mesos Run Modes Spark can run over Mesos in two modes: "fine-grained" and "coarse-grained". In fine-grained mode, which is the default, @@ -34,17 +35,15 @@ launch only *one* long-running Spark task on each Mesos machine, and dynamically it. The benefit is much lower startup overhead, but at the cost of reserving the Mesos resources for the complete duration of the application. -To run in coarse-grained mode, set the `spark.mesos.coarse` system property to true *before* creating your SparkContext: +To run in coarse-grained mode, set the `spark.mesos.coarse` property in your [SparkConf](configuration.html#spark-properties): {% highlight scala %} -System.setProperty("spark.mesos.coarse", "true") -val sc = new SparkContext("mesos://HOST:5050", "App Name", ...) +conf.set("spark.mesos.coarse", "true") {% endhighlight %} In addition, for coarse-grained mode, you can control the maximum number of resources Spark will acquire. By default, it will acquire *all* cores in the cluster (that get offered by Mesos), which only makes sense if you run just one -application at a time. You can cap the maximum number of cores using `System.setProperty("spark.cores.max", "10")` (for example). -Again, this must be done *before* initializing a SparkContext. +application at a time. You can cap the maximum number of cores using `conf.set("spark.cores.max", "10")` (for example). # Running Alongside Hadoop diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 56d2a3a4a0..1db255ca53 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -49,6 +49,9 @@ This is done through the following constructor: new SparkContext(master, appName, [sparkHome], [jars]) {% endhighlight %} +or through `new SparkContext(conf)`, which takes a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) +object for more advanced configuration. + 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, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `spark-shell` on four cores, use @@ -94,7 +97,6 @@ If you want to run your application on a cluster, you will need to specify the t 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) Spark revolves around the concept of a _resilient distributed dataset_ (RDD), which is a fault-tolerant collection of elements that can be operated on in parallel. There are currently two types of RDDs: *parallelized collections*, which take an existing Scala collection and run functions on it in parallel, and *Hadoop datasets*, which run functions on each record of a file in Hadoop distributed file system or any other storage system supported by Hadoop. Both types of RDDs can be operated on through the same methods. diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index b822265b5a..f7f0b78908 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -154,11 +154,18 @@ You can also pass an option `-c ` to control the number of cores that The standalone cluster mode currently only supports a simple FIFO scheduler across applications. However, to allow multiple concurrent users, you can control the maximum number of resources each -application will acquire. +application will use. By default, it will acquire *all* cores in the cluster, which only makes sense if you just run one -application at a time. You can cap the number of cores using -`System.setProperty("spark.cores.max", "10")` (for example). -This value must be set *before* initializing your SparkContext. +application at a time. You can cap the number of cores by setting `spark.cores.max` in your +[SparkConf](configuration.html#spark-properties). For example: + +{% highlight scala %} +val conf = new SparkConf() + .setMaster(...) + .setAppName(...) + .set("spark.cores.max", "10") +val sc = new SparkContext(conf) +{% endhighlight %} # Monitoring and Logging diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 82f42e0b8d..6f97db6a3e 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -326,7 +326,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 [`PairDStreamFunctions`](api/streaming/index.html#org.apache.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 [config property](configuration.html#spark-properties) `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. @@ -349,7 +349,7 @@ For a Spark Streaming application running on a cluster to be stable, the process A good approach to figure out the right batch size for your application is to test it with a conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (in the Spark master logs, find the line having the phrase "Total delay"). If the delay is maintained to be less than the batch size, then system is stable. Otherwise, if the delay is continuously increasing, it means that the system is unable to keep up and it therefore unstable. Once you have an idea of a stable configuration, you can try increasing the data rate and/or reducing the batch size. Note that momentary increase in the delay due to temporary data rate increases maybe fine as long as the delay reduces back to a low value (i.e., less than batch size). ## 24/7 Operation -By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the Java system property `spark.cleaner.ttl` to the number of seconds you want any metadata to persist. For example, setting `spark.cleaner.ttl` to 600 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created. +By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the [config property](configuration.html#spark-properties) `spark.cleaner.ttl` to the number of seconds you want any metadata to persist. For example, setting `spark.cleaner.ttl` to 600 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created. This value is closely tied with any window operation that is being used. Any window operation would require the input data to be persisted in memory for at least the duration of the window. Hence it is necessary to set the delay to at least the value of the largest window operation used in the Spark Streaming application. If this delay is set too low, the application will throw an exception saying so. diff --git a/docs/tuning.md b/docs/tuning.md index a4be188169..bbb870085c 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -38,14 +38,15 @@ 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.serializer.KryoSerializer")` -*before* creating your SparkContext. This setting configures the serializer used for not only shuffling data between worker +You can switch to using Kryo by initializing your job with a [SparkConf](configuration.html#spark-properties) +and calling `conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")`. +This setting configures the serializer used for not only shuffling data between worker nodes but also when serializing RDDs to disk. The only reason Kryo 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.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: +`spark.kryo.registrator` config property to point to it, as follows: {% highlight scala %} import com.esotericsoftware.kryo.Kryo @@ -58,17 +59,17 @@ class MyRegistrator extends KryoRegistrator { } } -// Make sure to set these properties *before* creating a SparkContext! -System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") -System.setProperty("spark.kryo.registrator", "mypackage.MyRegistrator") -val sc = new SparkContext(...) +val conf = new SparkConf().setMaster(...).setAppName(...) +conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") +conf.set("spark.kryo.registrator", "mypackage.MyRegistrator") +val sc = new SparkContext(conf) {% endhighlight %} The [Kryo documentation](http://code.google.com/p/kryo/) describes more advanced registration options, such as adding custom serialization code. If your objects are large, you may also need to increase the `spark.kryoserializer.buffer.mb` -system property. The default is 2, but this value needs to be large enough to hold the *largest* +config property. The default is 2, but this value needs to be large enough to hold the *largest* object you will serialize. Finally, if you don't register your classes, Kryo will still work, but it will have to store the @@ -165,7 +166,7 @@ cache RDDs. This means that 33% of memory is available for any objects created d 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 -`System.setProperty("spark.storage.memoryFraction", "0.5")`. Combined with the use of serialized caching, +`conf.set("spark.storage.memoryFraction", "0.5")` on your SparkConf. Combined with the use of serialized caching, using a smaller cache should be sufficient to mitigate most of the garbage collection problems. In case you are interested in further tuning the Java GC, continue reading below. @@ -219,7 +220,7 @@ enough. Spark automatically sets the number of "map" tasks to run on each file a 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.rdd.PairRDDFunctions) documentation), -or set the system property `spark.default.parallelism` to change the default. +or set the config property `spark.default.parallelism` to change the default. In general, we recommend 2-3 tasks per CPU core in your cluster. ## Memory Usage of Reduce Tasks diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index c07dd88307..9dcdcfaa67 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -44,6 +44,11 @@ u'/path' >>> conf.get("spark.executorEnv.VAR1") u'value1' +>>> print conf.toDebugString() +spark.executorEnv.VAR1=value1 +spark.executorEnv.VAR3=value3 +spark.executorEnv.VAR4=value4 +spark.home=/path >>> sorted(conf.getAll(), key=lambda p: p[0]) [(u'spark.executorEnv.VAR1', u'value1'), (u'spark.executorEnv.VAR3', u'value3'), (u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')] """ @@ -67,6 +72,9 @@ class SparkConf(object): All setter methods in this class support chaining. For example, you can write C{conf.setMaster("local").setAppName("My app")}. + + Note that once a SparkConf object is passed to Spark, it is cloned + and can no longer be modified by the user. """ def __init__(self, loadDefaults=True, _jvm=None): @@ -74,7 +82,9 @@ class SparkConf(object): Create a new Spark configuration. @param loadDefaults: whether to load values from Java system - properties and classpath (true by default) + properties and classpath (True by default) + @param _jvm: internal parameter used to pass a handle to the + Java VM; does not need to be set by users """ from pyspark.context import SparkContext SparkContext._ensure_initialized() @@ -97,10 +107,7 @@ class SparkConf(object): return self def setSparkHome(self, value): - """ - Set path where Spark is installed on worker nodes (needed for some - deployment modes). - """ + """Set path where Spark is installed on worker nodes.""" self._jconf.setSparkHome(value) return self @@ -144,6 +151,13 @@ class SparkConf(object): """Does this configuration contain a given key?""" return self._jconf.contains(key) + def toDebugString(self): + """ + Returns a printable version of the configuration, as a list of + key=value pairs, one per line. + """ + return self._jconf.toDebugString() + def _test(): import doctest diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 12ac0299e2..ee2f5cbe34 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -83,7 +83,7 @@ class SparkContext(object): SparkContext._ensure_initialized(self) self.environment = environment or {} - self.conf = conf or SparkConf(_jvm=self._jvm) + self._conf = conf or SparkConf(_jvm=self._jvm) self._batchSize = batchSize # -1 represents an unlimited batch size self._unbatched_serializer = serializer if batchSize == 1: @@ -94,31 +94,31 @@ class SparkContext(object): # Set parameters passed directly to us on the conf; these operations will be # no-ops if the parameters were None - self.conf.setMaster(master) - self.conf.setAppName(appName) - self.conf.setSparkHome(sparkHome) + self._conf.setMaster(master) + self._conf.setAppName(appName) + self._conf.setSparkHome(sparkHome) if environment: for key, value in environment.iteritems(): - self.conf.setExecutorEnv(key, value) + self._conf.setExecutorEnv(key, value) # Check that we have at least the required parameters - if not self.conf.contains("spark.master"): + if not self._conf.contains("spark.master"): raise Exception("A master URL must be set in your configuration") - if not self.conf.contains("spark.app.name"): + if not self._conf.contains("spark.app.name"): raise Exception("An application name must be set in your configuration") # Read back our properties from the conf in case we loaded some of them from # the classpath or an external config file - self.master = self.conf.get("spark.master") - self.appName = self.conf.get("spark.app.name") - self.sparkHome = self.conf.getOrElse("spark.home", None) - for (k, v) in self.conf.getAll(): + self.master = self._conf.get("spark.master") + self.appName = self._conf.get("spark.app.name") + self.sparkHome = self._conf.getOrElse("spark.home", None) + for (k, v) in self._conf.getAll(): if k.startswith("spark.executorEnv."): varName = k[len("spark.executorEnv."):] self.environment[varName] = v # Create the Java SparkContext through Py4J - self._jsc = self._jvm.JavaSparkContext(self.conf._jconf) + self._jsc = self._jvm.JavaSparkContext(self._conf._jconf) # Create a single Accumulator in Java that we'll send all our updates through; # they will be passed back to us through a TCP server -- cgit v1.2.3 From a7de8e9b1c9859f45db4a620dd62a62d472d8396 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Mon, 30 Dec 2013 19:28:03 -0800 Subject: Renamed countDistinct and countDistinctByKey methods to include Approx --- .../src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 10 +++++----- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- .../scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala | 6 +++--- core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala | 10 +++++----- .../org/apache/spark/serializer/KryoSerializerSuite.scala | 2 +- 5 files changed, 15 insertions(+), 15 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 4e4f860b19..1dc5f8d2f5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -217,7 +217,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * more accurate counts but increase the memory footprint and vise versa. Uses the provided * Partitioner to partition the output RDD. */ - def countDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = { + def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = { val createHLL = (v: V) => { val hll = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) hll.value.offer(v) @@ -242,8 +242,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * output RDD into numPartitions. * */ - def countDistinctByKey(relativeSD: Double, numPartitions: Int): RDD[(K, Long)] = { - countDistinctByKey(relativeSD, new HashPartitioner(numPartitions)) + def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): RDD[(K, Long)] = { + countApproxDistinctByKey(relativeSD, new HashPartitioner(numPartitions)) } /** @@ -254,8 +254,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism * level. */ - def countDistinctByKey(relativeSD: Double = 0.05): RDD[(K, Long)] = { - countDistinctByKey(relativeSD, defaultPartitioner(self)) + def countApproxDistinctByKey(relativeSD: Double = 0.05): RDD[(K, Long)] = { + countApproxDistinctByKey(relativeSD, defaultPartitioner(self)) } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 136fa45327..74fab48619 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -797,7 +797,7 @@ abstract class RDD[T: ClassTag]( * more accurate counts but increase the memory footprint and vise versa. The default value of * relativeSD is 0.05. */ - def countDistinct(relativeSD: Double = 0.05): Long = { + def countApproxDistinct(relativeSD: Double = 0.05): Long = { def hllCountPartition(iter: Iterator[T]): Iterator[SerializableHyperLogLog] = { val hllCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 6ad58b875d..5da538a1dd 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -110,7 +110,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { assert(deps.size === 2) // ShuffledRDD, ParallelCollection. } - test("countDistinctByKey") { + test("countApproxDistinctByKey") { def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble /* Since HyperLogLog unique counting is approximate, and the relative standard deviation is @@ -124,7 +124,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { // Therefore, the expected count for key i would be i. val stacked = (1 to 100).flatMap(i => (1 to i).map(j => (i, j))) val rdd1 = sc.parallelize(stacked) - val counted1 = rdd1.countDistinctByKey(relativeSD).collect() + val counted1 = rdd1.countApproxDistinctByKey(relativeSD).collect() counted1.foreach{ case(k, count) => assert(error(count, k) < relativeSD) } @@ -137,7 +137,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { (1 to num).map(j => (num, j)) } val rdd2 = sc.parallelize(randStacked) - val counted2 = rdd2.countDistinctByKey(relativeSD, 4).collect() + val counted2 = rdd2.countApproxDistinctByKey(relativeSD, 4).collect() counted2.foreach{ case(k, count) => assert(error(count, k) < relativeSD) } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 2f81b81797..1383359f85 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -63,17 +63,17 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } - test("countDistinct") { + test("countApproxDistinct") { def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble val size = 100 val uniformDistro = for (i <- 1 to 100000) yield i % size val simpleRdd = sc.makeRDD(uniformDistro) - assert(error(simpleRdd.countDistinct(0.2), size) < 0.2) - assert(error(simpleRdd.countDistinct(0.05), size) < 0.05) - assert(error(simpleRdd.countDistinct(0.01), size) < 0.01) - assert(error(simpleRdd.countDistinct(0.001), size) < 0.001) + assert(error(simpleRdd.countApproxDistinct(0.2), size) < 0.2) + assert(error(simpleRdd.countApproxDistinct(0.05), size) < 0.05) + assert(error(simpleRdd.countApproxDistinct(0.01), size) < 0.01) + assert(error(simpleRdd.countApproxDistinct(0.001), size) < 0.001) } test("SparkContext.union") { diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 18529710fe..636e3ab913 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -173,7 +173,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } test("kryo with SerializableHyperLogLog") { - assert(sc.parallelize( Array(1, 2, 3, 2, 3, 3, 2, 3, 1) ).countDistinct(0.01) === 3) + assert(sc.parallelize( Array(1, 2, 3, 2, 3, 3, 2, 3, 1) ).countApproxDistinct(0.01) === 3) } test("kryo with reduce") { -- cgit v1.2.3 From ed06500d300e93ae3129a035a364117adcb7d361 Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Mon, 30 Dec 2013 19:30:42 -0800 Subject: Added Java API for countApproxDistinctByKey --- .../org/apache/spark/api/java/JavaPairRDD.scala | 36 ++++++++++++++++++++++ 1 file changed, 36 insertions(+) (limited to 'core/src') 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 363667fa86..55c87450ac 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 @@ -611,6 +611,42 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K * Return an RDD with the values of each tuple. */ def values(): JavaRDD[V] = JavaRDD.fromRDD[V](rdd.map(_._2)) + + /** + * Return approximate number of distinct values for each key in this RDD. + * The accuracy of approximation can be controlled through the relative standard deviation + * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in + * more accurate counts but increase the memory footprint and vise versa. Uses the provided + * Partitioner to partition the output RDD. + */ + def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): JavaRDD[(K, Long)] = { + rdd.countApproxDistinctByKey(relativeSD, partitioner) + } + + /** + * Return approximate number of distinct values for each key this RDD. + * The accuracy of approximation can be controlled through the relative standard deviation + * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in + * more accurate counts but increase the memory footprint and vise versa. The default value of + * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism + * level. + */ + def countApproxDistinctByKey(relativeSD: Double = 0.05): JavaRDD[(K, Long)] = { + rdd.countApproxDistinctByKey(relativeSD) + } + + + /** + * Return approximate number of distinct values for each key in this RDD. + * The accuracy of approximation can be controlled through the relative standard deviation + * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in + * more accurate counts but increase the memory footprint and vise versa. HashPartitions the + * output RDD into numPartitions. + * + */ + def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaRDD[(K, Long)] = { + rdd.countApproxDistinctByKey(relativeSD, numPartitions) + } } object JavaPairRDD { -- cgit v1.2.3 From c3073b6cf2a647451441e8dfc18fe4334497113c Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Mon, 30 Dec 2013 19:31:06 -0800 Subject: Added Java API for countApproxDistinct --- .../main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) (limited to 'core/src') 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 f344804b4c..924d8af060 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 @@ -444,4 +444,15 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[T]] takeOrdered(num, comp) } + + /** + * Return approximate number of distinct elements in the RDD. + * + * The accuracy of approximation can be controlled through the relative standard deviation + * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in + * more accurate counts but increase the memory footprint and vise versa. The default value of + * relativeSD is 0.05. + */ + def countApproxDistinct(relativeSD: Double = 0.05): Long = rdd.countApproxDistinct(relativeSD) + } -- cgit v1.2.3 From d6cded7155b36880f81544bdf6fc6c20dd52ad7d Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Mon, 30 Dec 2013 19:32:05 -0800 Subject: Added Java unit tests for countApproxDistinct and countApproxDistinctByKey --- .../test/scala/org/apache/spark/JavaAPISuite.java | 32 ++++++++++++++++++++++ 1 file changed, 32 insertions(+) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java index 79913dc718..6398feb9f8 100644 --- a/core/src/test/scala/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java @@ -930,4 +930,36 @@ public class JavaAPISuite implements Serializable { parts[1]); } + @Test + public void countApproxDistinct() { + List arrayData = new ArrayList(); + int size = 100; + for (int i = 0; i < 100000; i++) { + arrayData.add(i % size); + } + JavaRDD simpleRdd = sc.parallelize(arrayData, 10); + Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.2) - size) / (size * 1.0)) < 0.2); + Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.05) - size) / (size * 1.0)) <= 0.05); + Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.01) - size) / (size * 1.0)) <= 0.01); + } + + @Test + public void countApproxDistinctByKey() { + double relativeSD = 0.001; + + List> arrayData = new ArrayList>(); + for (int i = 10; i < 100; i++) + for (int j = 0; j < i; j++) + arrayData.add(new Tuple2(i, j)); + + JavaPairRDD pairRdd = sc.parallelizePairs(arrayData); + List> res = pairRdd.countApproxDistinctByKey(relativeSD).collect(); + for (Tuple2 resItem : res) { + double count = (double)resItem._1(); + Long resCount = (Long)resItem._2(); + Double error = Math.abs((resCount - count) / count); + Assert.assertTrue(error < relativeSD); + } + + } } -- cgit v1.2.3 From 18181e6c4120b04d125aa99a1ac63e5e7b2c0e3d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 30 Dec 2013 23:37:41 -0800 Subject: Removing initLogging entirely --- .../main/scala/org/apache/spark/HttpServer.scala | 1 + core/src/main/scala/org/apache/spark/Logging.scala | 23 +++++++++++++++++++--- .../main/scala/org/apache/spark/SparkContext.scala | 3 --- .../scala/org/apache/spark/executor/Executor.scala | 2 -- .../org/apache/spark/metrics/MetricsConfig.scala | 1 - .../org/apache/spark/metrics/MetricsSystem.scala | 1 - .../spark/storage/BlockManagerMasterActor.scala | 2 -- .../apache/spark/storage/BlockManagerWorker.scala | 3 --- .../apache/spark/storage/BlockMessageArray.scala | 2 -- .../scala/org/apache/spark/repl/SparkILoop.scala | 2 -- .../scala/org/apache/spark/streaming/DStream.scala | 2 -- .../org/apache/spark/streaming/DStreamGraph.scala | 1 - .../apache/spark/streaming/StreamingContext.scala | 2 -- .../streaming/dstream/NetworkInputDStream.scala | 2 -- .../spark/streaming/scheduler/JobGenerator.scala | 1 - .../spark/streaming/scheduler/JobScheduler.scala | 2 -- .../spark/streaming/util/MasterFailureTest.scala | 3 --- 17 files changed, 21 insertions(+), 32 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index cdfc9dd54e..240f32efaf 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -46,6 +46,7 @@ private[spark] class HttpServer(resourceBase: File) extends Logging { if (server != null) { throw new ServerStateException("Server is already started") } else { + log.info("Starting HTTP Server") server = new Server() val connector = new SocketConnector connector.setMaxIdleTime(60*1000) diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index b97697d587..1fdbccdec6 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -33,6 +33,7 @@ trait Logging { // Method to get or create the logger for this object protected def log: Logger = { if (log_ == null) { + initializeIfNecessary() var className = this.getClass.getName // Ignore trailing $'s in the class names for Scala objects if (className.endsWith("$")) { @@ -89,9 +90,15 @@ trait Logging { log.isTraceEnabled } - // Method for ensuring that logging is initialized, to avoid having multiple - // threads do it concurrently (as SLF4J initialization is not thread safe). - protected def initLogging() { + private def initializeIfNecessary() { + Logging.initLock.synchronized { + if (!Logging.initialized) { + initializeLogging() + } + } + } + + private def initializeLogging() { // If Log4j doesn't seem initialized, load a default properties file val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements if (!log4jInitialized) { @@ -101,7 +108,17 @@ trait Logging { case Some(url) => PropertyConfigurator.configure(url) case None => System.err.println(s"Spark was unable to load $defaultLogProps") } + log.info(s"Using Spark's default log4j profile: $defaultLogProps") } + Logging.initialized = true + + // Force a call into slf4j to initialize it avoids this happening from mutliple threads + // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html log } } + +object Logging { + @transient @volatile private var initialized = false + @transient val initLock = new Object() +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ad3337d94c..70fd499d71 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -88,9 +88,6 @@ class SparkContext( scala.collection.immutable.Map()) extends Logging { - // Ensure logging is initialized before we spawn any threads - initLogging() - // Set Spark driver host and port system properties if (System.getProperty("spark.driver.host") == null) { System.setProperty("spark.driver.host", Utils.localHostName()) 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 0f19d7a96b..782be9a429 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -48,8 +48,6 @@ private[spark] class Executor( private val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0)) - initLogging() - // No ip or host:port - just hostname Utils.checkHost(slaveHostname, "Expected executed slave to be a hostname") // must not have port specified. diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index caab748d60..6f9f29969e 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -26,7 +26,6 @@ import scala.util.matching.Regex import org.apache.spark.Logging private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging { - initLogging() val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index bec0c83be8..8e038ce98c 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -63,7 +63,6 @@ import org.apache.spark.metrics.source.Source * [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") val metricsConfig = new MetricsConfig(Option(confFile)) 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 21022e1cfb..e0eb02ce81 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -50,8 +50,6 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { private val akkaTimeout = AkkaUtils.askTimeout - initLogging() - val slaveTimeout = System.getProperty("spark.storage.blockManagerSlaveTimeoutMs", "" + (BlockManager.getHeartBeatFrequencyFromSystemProperties * 3)).toLong 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 0c66addf9d..21f003609b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala @@ -30,7 +30,6 @@ import org.apache.spark.util.Utils * TODO: Use event model. */ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends Logging { - initLogging() blockManager.connectionManager.onReceiveMessage(onBlockMessageReceive) @@ -101,8 +100,6 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends private[spark] object BlockManagerWorker extends Logging { private var blockManagerWorker: BlockManagerWorker = null - initLogging() - def startBlockManagerWorker(manager: BlockManager) { blockManagerWorker = new BlockManagerWorker(manager) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala index 6ce9127c74..a06f50a0ac 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala @@ -37,8 +37,6 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockM def length = blockMessages.length - initLogging() - def set(bufferMessage: BufferMessage) { val startTime = System.currentTimeMillis val newBlockMessages = new ArrayBuffer[BlockMessage]() diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index b2a1815459..523fd1222d 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -60,8 +60,6 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out, None) def this() = this(None, new JPrintWriter(Console.out, true), None) - initLogging() - var in: InteractiveReader = _ // the input stream from which commands come var settings: Settings = _ var intp: SparkIMain = _ 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 a78d3965ee..8ebe09da0d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala @@ -56,8 +56,6 @@ abstract class DStream[T: ClassTag] ( @transient protected[streaming] var ssc: StreamingContext ) extends Serializable with Logging { - initLogging() - // ======================================================================= // Methods that should be implemented by subclasses of DStream // ======================================================================= diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index daed7ff7c3..a09b891956 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -24,7 +24,6 @@ import org.apache.spark.Logging import org.apache.spark.streaming.scheduler.Job final private[streaming] class DStreamGraph extends Serializable with Logging { - initLogging() private val inputStreams = new ArrayBuffer[InputDStream[_]]() private val outputStreams = new ArrayBuffer[DStream[_]]() 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 41da028a3c..c759b36f94 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -95,8 +95,6 @@ class StreamingContext private ( */ def this(path: String) = this(null, CheckpointReader.read(path), null) - initLogging() - if (sc_ == null && cp_ == null) { throw new Exception("Spark Streaming cannot be initialized with " + "both SparkContext and checkpoint as null") 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 5add20871e..8c7f42306d 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 @@ -88,8 +88,6 @@ private[streaming] case class ReportError(msg: String) extends NetworkReceiverMe */ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging { - initLogging() - lazy protected val env = SparkEnv.get lazy protected val actor = env.actorSystem.actorOf( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 1cd0b9b0a4..afe9316337 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -29,7 +29,6 @@ import org.apache.spark.streaming.util.{ManualClock, RecurringTimer, Clock} private[streaming] class JobGenerator(jobScheduler: JobScheduler) extends Logging { - initLogging() val ssc = jobScheduler.ssc val clockClass = System.getProperty( "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 9511ccfbed..488cc2f401 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -30,8 +30,6 @@ import org.apache.spark.streaming._ private[streaming] class JobScheduler(val ssc: StreamingContext) extends Logging { - initLogging() - val jobSets = new ConcurrentHashMap[Time, JobSet] val numConcurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt val executor = Executors.newFixedThreadPool(numConcurrentJobs) 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 4a3993e3e3..1559f7a9f7 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 @@ -39,7 +39,6 @@ import org.apache.hadoop.conf.Configuration private[streaming] object MasterFailureTest extends Logging { - initLogging() @volatile var killed = false @volatile var killCount = 0 @@ -331,7 +330,6 @@ class TestOutputStream[T: ClassTag]( */ private[streaming] class KillingThread(ssc: StreamingContext, maxKillWaitTime: Long) extends Thread with Logging { - initLogging() override def run() { try { @@ -366,7 +364,6 @@ class KillingThread(ssc: StreamingContext, maxKillWaitTime: Long) extends Thread private[streaming] class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) extends Thread with Logging { - initLogging() override def run() { val localTestDir = Files.createTempDir() -- cgit v1.2.3 From 3c254f2eec87fbb9de4589816102245e01b0d42c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 30 Dec 2013 23:55:33 -0800 Subject: Minor fixes --- core/src/main/scala/org/apache/spark/HttpServer.scala | 2 +- core/src/main/scala/org/apache/spark/Logging.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index 240f32efaf..69a738dc44 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -46,7 +46,7 @@ private[spark] class HttpServer(resourceBase: File) extends Logging { if (server != null) { throw new ServerStateException("Server is already started") } else { - log.info("Starting HTTP Server") + logInfo("Starting HTTP Server") server = new Server() val connector = new SocketConnector connector.setMaxIdleTime(60*1000) diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 1fdbccdec6..6054adb52e 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -119,6 +119,6 @@ trait Logging { } object Logging { - @transient @volatile private var initialized = false - @transient val initLock = new Object() + @volatile private var initialized = false + val initLock = new Object() } \ No newline at end of file -- cgit v1.2.3 From 4d009dcac61776e455e7757ebf61630614b16fde Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 31 Dec 2013 00:01:44 -0800 Subject: Removing use in test --- core/src/test/scala/org/apache/spark/CheckpointSuite.scala | 2 -- 1 file changed, 2 deletions(-) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index f25d921d3f..70bfb81661 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -26,8 +26,6 @@ import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId} import org.apache.spark.util.Utils class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { - initLogging() - var checkpointDir: File = _ val partitioner = new HashPartitioner(2) -- cgit v1.2.3 From 4abb0c57ab43de9461518e255dfec5e113d88298 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 31 Dec 2013 00:05:03 -0800 Subject: Tiny typo fix --- core/src/main/scala/org/apache/spark/Logging.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 6054adb52e..38df3fe04b 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -112,7 +112,7 @@ trait Logging { } Logging.initialized = true - // Force a call into slf4j to initialize it avoids this happening from mutliple threads + // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html log } @@ -121,4 +121,4 @@ trait Logging { object Logging { @volatile private var initialized = false val initLock = new Object() -} \ No newline at end of file +} -- cgit v1.2.3 From fcd17a1e8ef1d0f106e845f4de99533d61cd8695 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 31 Dec 2013 02:01:45 -0800 Subject: Fixed comments and long lines based on comments on PR 289. --- .../main/scala/org/apache/spark/rdd/RDDCheckpointData.scala | 3 ++- .../main/scala/org/apache/spark/streaming/Checkpoint.scala | 4 ++-- .../scala/org/apache/spark/streaming/StreamingContext.scala | 9 +++++++-- .../spark/streaming/api/java/JavaStreamingContext.scala | 13 ++++++++----- 4 files changed, 19 insertions(+), 10 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index 091a6fdb54..642dabaad5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -91,7 +91,8 @@ private[spark] class RDDCheckpointData[T: ClassTag](rdd: RDD[T]) } // Save to file, and reload it as an RDD - val broadcastedConf = rdd.context.broadcast(new SerializableWritable(rdd.context.hadoopConfiguration)) + val broadcastedConf = rdd.context.broadcast( + new SerializableWritable(rdd.context.hadoopConfiguration)) rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path.toString, broadcastedConf) _) val newRDD = new CheckpointRDD[T](rdd.context, path.toString) if (newRDD.partitions.size != rdd.partitions.size) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 0f9a71983e..4960a85b97 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -81,8 +81,8 @@ class CheckpointWriter(checkpointDir: String, hadoopConf: Configuration) extends attempts += 1 try { logDebug("Saving checkpoint for time " + checkpointTime + " to file '" + file + "'") - // This is inherently thread unsafe .. so alleviating it by writing to '.new' and - // then doing moves : which should be pretty fast. + // This is inherently thread unsafe, so alleviating it by writing to '.new' and + // then moving it to the final file val fos = fs.create(writeFile) fos.write(bytes) fos.close() 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 6628fdcc85..8898fdcb7f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -371,7 +371,8 @@ class StreamingContext private ( /** * Create a input stream that monitors a Hadoop-compatible filesystem * for new files and reads them using the given key-value types and input format. - * File names starting with . are ignored. + * Files must be written to the monitored directory by "moving" them from another + * location within the same file system. File names starting with . are ignored. * @param directory HDFS directory to monitor for new file * @tparam K Key type for reading HDFS file * @tparam V Value type for reading HDFS file @@ -390,6 +391,8 @@ class StreamingContext private ( /** * Create a input stream that monitors a Hadoop-compatible filesystem * for new files and reads them using the given key-value types and input format. + * Files must be written to the monitored directory by "moving" them from another + * location within the same file system. * @param directory HDFS directory to monitor for new file * @param filter Function to filter paths to process * @param newFilesOnly Should process only new files and ignore existing files in the directory @@ -410,7 +413,9 @@ class StreamingContext private ( /** * Create a input stream that monitors a Hadoop-compatible filesystem * for new files and reads them as text files (using key as LongWritable, value - * as Text and input format as TextInputFormat). File names starting with . are ignored. + * as Text and input format as TextInputFormat). Files must be written to the + * monitored directory by "moving" them from another location within the same + * file system. File names starting with . are ignored. * @param directory HDFS directory to monitor for new file */ def textFileStream(directory: String): DStream[String] = { 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 78d318cf27..aad0d931e7 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 @@ -256,9 +256,11 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Creates a input stream that monitors a Hadoop-compatible filesystem + * Create a input stream that monitors a Hadoop-compatible filesystem * for new files and reads them as text files (using key as LongWritable, value - * as Text and input format as TextInputFormat). File names starting with . are ignored. + * as Text and input format as TextInputFormat). Files must be written to the + * monitored directory by "moving" them from another location within the same + * file system. File names starting with . are ignored. * @param directory HDFS directory to monitor for new file */ def textFileStream(directory: String): JavaDStream[String] = { @@ -300,9 +302,10 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Creates a input stream that monitors a Hadoop-compatible filesystem + * Create a input stream that monitors a Hadoop-compatible filesystem * for new files and reads them using the given key-value types and input format. - * File names starting with . are ignored. + * Files must be written to the monitored directory by "moving" them from another + * location within the same file system. File names starting with . are ignored. * @param directory HDFS directory to monitor for new file * @tparam K Key type for reading HDFS file * @tparam V Value type for reading HDFS file @@ -331,7 +334,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** - * Creates a input stream from a Flume source. + * Create a input stream from a Flume source. * @param hostname Hostname of the slave machine to which the flume data will be sent * @param port Port of the slave machine to which the flume data will be sent */ -- cgit v1.2.3 From acb0323053d270a377e497e975b2dfe59e2f997c Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Tue, 31 Dec 2013 15:34:26 -0800 Subject: minor improvements --- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 5 ++--- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 4 +++- 2 files changed, 5 insertions(+), 4 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 1dc5f8d2f5..088b298aad 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -229,9 +229,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) } val mergeHLL = (h1: SerializableHyperLogLog, h2: SerializableHyperLogLog) => h1.merge(h2) - combineByKey(createHLL, mergeValueHLL, mergeHLL, partitioner).map { - case (k, v) => (k, v.value.cardinality()) - } + combineByKey(createHLL, mergeValueHLL, mergeHLL, partitioner).mapValues(_.value.cardinality()) + } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 74fab48619..161fd067e1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -809,7 +809,9 @@ abstract class RDD[T: ClassTag]( } def mergeCounters(c1: SerializableHyperLogLog, c2: SerializableHyperLogLog) = c1.merge(c2) - mapPartitions(hllCountPartition).reduce(mergeCounters).value.cardinality() + val zeroCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) + mapPartitions(hllCountPartition).aggregate(zeroCounter)(mergeCounters, mergeCounters) + .value.cardinality() } /** -- cgit v1.2.3 From bee445c927586136673f39259f23642a5a6e8efe Mon Sep 17 00:00:00 2001 From: Hossein Falaki Date: Tue, 31 Dec 2013 16:58:18 -0800 Subject: Made the code more compact and readable --- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 12 ++---------- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 14 +------------- .../org/apache/spark/util/SerializableHyperLogLog.scala | 5 +++++ 3 files changed, 8 insertions(+), 23 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 088b298aad..04a8d05988 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -218,19 +218,11 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Partitioner to partition the output RDD. */ def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = { - val createHLL = (v: V) => { - val hll = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) - hll.value.offer(v) - hll - } - val mergeValueHLL = (hll: SerializableHyperLogLog, v: V) => { - hll.value.offer(v) - hll - } + val createHLL = (v: V) => new SerializableHyperLogLog(new HyperLogLog(relativeSD)).add(v) + val mergeValueHLL = (hll: SerializableHyperLogLog, v: V) => hll.add(v) val mergeHLL = (h1: SerializableHyperLogLog, h2: SerializableHyperLogLog) => h1.merge(h2) combineByKey(createHLL, mergeValueHLL, mergeHLL, partitioner).mapValues(_.value.cardinality()) - } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 161fd067e1..4960e6e82f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -798,20 +798,8 @@ abstract class RDD[T: ClassTag]( * relativeSD is 0.05. */ def countApproxDistinct(relativeSD: Double = 0.05): Long = { - - def hllCountPartition(iter: Iterator[T]): Iterator[SerializableHyperLogLog] = { - val hllCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) - while (iter.hasNext) { - val v = iter.next() - hllCounter.value.offer(v) - } - Iterator(hllCounter) - } - def mergeCounters(c1: SerializableHyperLogLog, c2: SerializableHyperLogLog) = c1.merge(c2) - val zeroCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) - mapPartitions(hllCountPartition).aggregate(zeroCounter)(mergeCounters, mergeCounters) - .value.cardinality() + aggregate(zeroCounter)(_.add(_), _.merge(_)).value.cardinality() } /** diff --git a/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala index 9cfd41407f..8b4e7c104c 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala @@ -30,6 +30,11 @@ class SerializableHyperLogLog(var value: ICardinality) extends Externalizable { def merge(other: SerializableHyperLogLog) = new SerializableHyperLogLog(value.merge(other.value)) + def add[T](elem: T) = { + this.value.offer(elem) + this + } + def readExternal(in: ObjectInput) { val byteLength = in.readInt() val bytes = new Array[Byte](byteLength) -- cgit v1.2.3 From 37c43c9dd1947e7cc99a310214b323707948f087 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 31 Dec 2013 17:36:56 -0800 Subject: Adding outer checkout when initializing logging --- core/src/main/scala/org/apache/spark/Logging.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 38df3fe04b..d519fc5a29 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -91,9 +91,11 @@ trait Logging { } private def initializeIfNecessary() { - Logging.initLock.synchronized { - if (!Logging.initialized) { - initializeLogging() + if (!Logging.initialized) { + Logging.initLock.synchronized { + if (!Logging.initialized) { + initializeLogging() + } } } } -- cgit v1.2.3 From e2c68642c64345434e2034082cf9b299491e9e9f Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 1 Jan 2014 22:03:39 -0500 Subject: Miscellaneous fixes from code review. Also replaced SparkConf.getOrElse with just a "get" that takes a default value, and added getInt, getLong, etc to make code that uses this simpler later on. --- .../main/scala/org/apache/spark/SparkConf.scala | 60 +++++++++++------- .../main/scala/org/apache/spark/SparkContext.scala | 73 +++++++++++++--------- .../src/main/scala/org/apache/spark/SparkEnv.scala | 10 +-- .../org/apache/spark/api/python/PythonRDD.scala | 4 +- .../org/apache/spark/broadcast/Broadcast.scala | 4 +- .../org/apache/spark/broadcast/HttpBroadcast.scala | 4 +- .../apache/spark/broadcast/TorrentBroadcast.scala | 2 +- .../org/apache/spark/deploy/master/Master.scala | 12 ++-- .../spark/deploy/master/MasterArguments.scala | 6 +- .../deploy/master/SparkZooKeeperSession.scala | 2 +- .../master/ZooKeeperLeaderElectionAgent.scala | 2 +- .../deploy/master/ZooKeeperPersistenceEngine.scala | 2 +- .../org/apache/spark/deploy/worker/Worker.scala | 2 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- .../scala/org/apache/spark/executor/Executor.scala | 2 +- .../org/apache/spark/io/CompressionCodec.scala | 4 +- .../org/apache/spark/metrics/MetricsSystem.scala | 2 +- .../apache/spark/network/ConnectionManager.scala | 18 +++--- .../apache/spark/network/netty/ShuffleCopier.scala | 2 +- .../scala/org/apache/spark/rdd/CheckpointRDD.scala | 4 +- .../apache/spark/scheduler/TaskResultGetter.scala | 2 +- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 10 +-- .../apache/spark/scheduler/TaskSetManager.scala | 16 ++--- .../cluster/CoarseGrainedSchedulerBackend.scala | 9 +-- .../scheduler/cluster/SimrSchedulerBackend.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../mesos/CoarseMesosSchedulerBackend.scala | 4 +- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- .../apache/spark/serializer/KryoSerializer.scala | 13 ++-- .../spark/storage/BlockFetcherIterator.scala | 2 +- .../org/apache/spark/storage/BlockManager.scala | 24 +++---- .../apache/spark/storage/BlockManagerMaster.scala | 4 +- .../spark/storage/BlockManagerMasterActor.scala | 4 +- .../apache/spark/storage/DiskBlockManager.scala | 2 +- .../apache/spark/storage/ShuffleBlockManager.scala | 6 +- .../main/scala/org/apache/spark/ui/SparkUI.scala | 4 +- .../org/apache/spark/ui/env/EnvironmentUI.scala | 2 +- .../apache/spark/ui/jobs/JobProgressListener.scala | 2 +- .../scala/org/apache/spark/util/AkkaUtils.scala | 18 +++--- .../org/apache/spark/util/MetadataCleaner.scala | 4 +- .../main/scala/org/apache/spark/util/Utils.scala | 4 +- .../scala/org/apache/spark/SparkConfSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 2 +- .../spark/storage/DiskBlockManagerSuite.scala | 12 +--- python/pyspark/conf.py | 12 ++-- .../scala/org/apache/spark/repl/SparkIMain.scala | 2 +- .../streaming/dstream/NetworkInputDStream.scala | 6 +- .../spark/streaming/scheduler/JobGenerator.scala | 4 +- .../spark/streaming/scheduler/JobScheduler.scala | 2 +- 49 files changed, 206 insertions(+), 189 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 96239cf4be..98343e9532 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -42,6 +42,12 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { /** Set a configuration variable. */ def set(key: String, value: String): SparkConf = { + if (key == null) { + throw new NullPointerException("null key") + } + if (value == null) { + throw new NullPointerException("null value") + } settings(key) = value this } @@ -51,26 +57,17 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster. */ def setMaster(master: String): SparkConf = { - if (master != null) { - settings("spark.master") = master - } - this + set("spark.master", master) } /** Set a name for your application. Shown in the Spark web UI. */ def setAppName(name: String): SparkConf = { - if (name != null) { - settings("spark.app.name") = name - } - this + set("spark.app.name", name) } /** Set JAR files to distribute to the cluster. */ def setJars(jars: Seq[String]): SparkConf = { - if (!jars.isEmpty) { - settings("spark.jars") = jars.mkString(",") - } - this + set("spark.jars", jars.mkString(",")) } /** Set JAR files to distribute to the cluster. (Java-friendly version.) */ @@ -84,8 +81,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { * (for example spark.executorEnv.PATH) but this method makes them easier to set. */ def setExecutorEnv(variable: String, value: String): SparkConf = { - settings("spark.executorEnv." + variable) = value - this + set("spark.executorEnv." + variable, value) } /** @@ -112,10 +108,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { * Set the location where Spark is installed on worker nodes. */ def setSparkHome(home: String): SparkConf = { - if (home != null) { - settings("spark.home") = home - } - this + set("spark.home", home) } /** Set multiple parameters together */ @@ -132,9 +125,20 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { this } - /** Get a parameter; throws an exception if it's not set */ + /** Remove a parameter from the configuration */ + def remove(key: String): SparkConf = { + settings.remove(key) + this + } + + /** Get a parameter; throws a NoSuchElementException if it's not set */ def get(key: String): String = { - settings(key) + settings.getOrElse(key, throw new NoSuchElementException(key)) + } + + /** Get a parameter, falling back to a default if not set */ + def get(key: String, defaultValue: String): String = { + settings.getOrElse(key, defaultValue) } /** Get a parameter as an Option */ @@ -145,9 +149,19 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { /** Get all parameters as a list of pairs */ def getAll: Array[(String, String)] = settings.clone().toArray - /** Get a parameter, falling back to a default if not set */ - def getOrElse(k: String, defaultValue: String): String = { - settings.getOrElse(k, defaultValue) + /** Get a parameter as an integer, falling back to a default if not set */ + def getInt(key: String, defaultValue: Int): Int = { + getOption(key).map(_.toInt).getOrElse(defaultValue) + } + + /** Get a parameter as a long, falling back to a default if not set */ + def getLong(key: String, defaultValue: Long): Long = { + getOption(key).map(_.toLong).getOrElse(defaultValue) + } + + /** Get a parameter as a double, falling back to a default if not set */ + def getDouble(key: String, defaultValue: Double): Double = { + getOption(key).map(_.toDouble).getOrElse(defaultValue) } /** Get all executor environment variables set on this SparkConf */ diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 46874c41a2..84bd0f7ffd 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -22,12 +22,11 @@ import java.net.URI import java.util.{UUID, Properties} import java.util.concurrent.atomic.AtomicInteger -import scala.collection.{Map, Set, immutable} +import scala.collection.{Map, Set} import scala.collection.generic.Growable import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.reflect.{ClassTag, classTag} -import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -49,7 +48,8 @@ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, Me import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI -import org.apache.spark.util._ +import org.apache.spark.util.{Utils, TimeStampedHashMap, MetadataCleaner, MetadataCleanerType, +ClosureCleaner} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -77,7 +77,7 @@ class SparkContext( * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters */ def this(master: String, appName: String, conf: SparkConf) = - this(conf.clone().setMaster(master).setAppName(appName)) + this(SparkContext.updatedConf(conf, master, appName)) /** * Alternative constructor that allows setting common Spark properties directly @@ -97,13 +97,7 @@ class SparkContext( environment: Map[String, String] = Map(), preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) = { - this( - new SparkConf() - .setMaster(master) - .setAppName(appName) - .setJars(jars) - .setExecutorEnv(environment.toSeq) - .setSparkHome(sparkHome), + this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment), preferredNodeLocationData) } @@ -175,11 +169,9 @@ 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_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) { - val value = System.getenv(key) - if (value != null) { - executorEnvs(key) = value - } + for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING"); + value <- Option(System.getenv(key))) { + executorEnvs(key) = value } // Since memory can be set with a system property too, use that executorEnvs("SPARK_MEM") = executorMemory + "m" @@ -220,7 +212,7 @@ class SparkContext( hadoopConf.set(key.substring("spark.hadoop.".length), value) } } - val bufferSize = conf.getOrElse("spark.buffer.size", "65536") + val bufferSize = conf.get("spark.buffer.size", "65536") hadoopConf.set("io.file.buffer.size", bufferSize) hadoopConf } @@ -733,13 +725,7 @@ class SparkContext( * (in that order of preference). If neither of these is set, return None. */ private[spark] def getSparkHome(): Option[String] = { - if (conf.contains("spark.home")) { - Some(conf.get("spark.home")) - } else if (System.getenv("SPARK_HOME") != null) { - Some(System.getenv("SPARK_HOME")) - } else { - None - } + conf.getOption("spark.home").orElse(Option(System.getenv("SPARK_HOME"))) } /** @@ -1026,7 +1012,7 @@ object SparkContext { /** * Find the JAR from which a given class was loaded, to make it easy for users to pass - * their JARs to SparkContext + * their JARs to SparkContext. */ def jarOfClass(cls: Class[_]): Seq[String] = { val uri = cls.getResource("/" + cls.getName.replace('.', '/') + ".class") @@ -1043,10 +1029,41 @@ object SparkContext { } } - /** Find the JAR that contains the class of a particular object */ + /** + * Find the JAR that contains the class of a particular object, to make it easy for users + * to pass their JARs to SparkContext. In most cases you can call jarOfObject(this) in + * your driver program. + */ def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass) - // Creates a task scheduler based on a given master URL. Extracted for testing. + /** + * Creates a modified version of a SparkConf with the parameters that can be passed separately + * to SparkContext, to make it easier to write SparkContext's constructors. This ignores + * parameters that are passed as the default value of null, instead of throwing an exception + * like SparkConf would. + */ + private def updatedConf( + conf: SparkConf, + master: String, + appName: String, + sparkHome: String = null, + jars: Seq[String] = Nil, + environment: Map[String, String] = Map()): SparkConf = + { + val res = conf.clone() + res.setMaster(master) + res.setAppName(appName) + if (sparkHome != null) { + res.setSparkHome(sparkHome) + } + if (!jars.isEmpty) { + res.setJars(jars) + } + res.setExecutorEnv(environment.toSeq) + res + } + + /** Creates a task scheduler based on a given master URL. Extracted for testing. */ private def createTaskScheduler(sc: SparkContext, master: String, appName: String) : TaskScheduler = { @@ -1156,7 +1173,7 @@ object SparkContext { case mesosUrl @ MESOS_REGEX(_) => MesosNativeLibrary.load() val scheduler = new TaskSchedulerImpl(sc) - val coarseGrained = sc.conf.getOrElse("spark.mesos.coarse", "false").toBoolean + val coarseGrained = sc.conf.get("spark.mesos.coarse", "false").toBoolean val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { new CoarseMesosSchedulerBackend(scheduler, sc, url, appName) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index d06af8e667..634a94f0a7 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -144,17 +144,17 @@ object SparkEnv extends Logging { // Create an instance of the class named by the given Java system property, or by // defaultClassName if the property is not set, and return it as a T def instantiateClass[T](propertyName: String, defaultClassName: String): T = { - val name = conf.getOrElse(propertyName, defaultClassName) + val name = conf.get(propertyName, defaultClassName) Class.forName(name, true, classLoader).newInstance().asInstanceOf[T] } val serializerManager = new SerializerManager val serializer = serializerManager.setDefault( - conf.getOrElse("spark.serializer", "org.apache.spark.serializer.JavaSerializer"), conf) + conf.get("spark.serializer", "org.apache.spark.serializer.JavaSerializer"), conf) val closureSerializer = serializerManager.get( - conf.getOrElse("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"), + conf.get("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"), conf) def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = { @@ -162,8 +162,8 @@ object SparkEnv extends Logging { logInfo("Registering " + name) Left(actorSystem.actorOf(Props(newActor), name = name)) } else { - val driverHost: String = conf.getOrElse("spark.driver.host", "localhost") - val driverPort: Int = conf.getOrElse("spark.driver.port", "7077").toInt + val driverHost: String = conf.get("spark.driver.host", "localhost") + val driverPort: Int = conf.get("spark.driver.port", "7077").toInt Utils.checkHost(driverHost, "Expected hostname") val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name) logInfo("Connecting to " + name + ": " + url) 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 05fd824254..32cc70e8c9 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 @@ -41,7 +41,7 @@ private[spark] class PythonRDD[T: ClassTag]( accumulator: Accumulator[JList[Array[Byte]]]) extends RDD[Array[Byte]](parent) { - val bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = conf.get("spark.buffer.size", "65536").toInt override def getPartitions = parent.partitions @@ -250,7 +250,7 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: Utils.checkHost(serverHost, "Expected hostname") - val bufferSize = SparkEnv.get.conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = SparkEnv.get.conf.get("spark.buffer.size", "65536").toInt override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index be99d229ef..0fc478a419 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -31,7 +31,7 @@ abstract class Broadcast[T](private[spark] val id: Long) extends Serializable { override def toString = "Broadcast(" + id + ")" } -private[spark] +private[spark] class BroadcastManager(val _isDriver: Boolean, conf: SparkConf) extends Logging with Serializable { private var initialized = false @@ -43,7 +43,7 @@ class BroadcastManager(val _isDriver: Boolean, conf: SparkConf) extends Logging private def initialize() { synchronized { if (!initialized) { - val broadcastFactoryClass = conf.getOrElse( + val broadcastFactoryClass = conf.get( "spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") broadcastFactory = 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 47528bcee8..db596d5fcc 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -92,8 +92,8 @@ private object HttpBroadcast extends Logging { def initialize(isDriver: Boolean, conf: SparkConf) { synchronized { if (!initialized) { - bufferSize = conf.getOrElse("spark.buffer.size", "65536").toInt - compress = conf.getOrElse("spark.broadcast.compress", "true").toBoolean + bufferSize = conf.get("spark.buffer.size", "65536").toInt + compress = conf.get("spark.broadcast.compress", "true").toBoolean if (isDriver) { createServer(conf) conf.set("spark.httpBroadcast.uri", serverUri) diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 00ec3b971b..9530938278 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -180,7 +180,7 @@ extends Logging { initialized = false } - lazy val BLOCK_SIZE = conf.getOrElse("spark.broadcast.blockSize", "4096").toInt * 1024 + lazy val BLOCK_SIZE = conf.get("spark.broadcast.blockSize", "4096").toInt * 1024 def blockifyObject[T](obj: T): TorrentInfo = { val byteArray = Utils.serialize[T](obj) 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 9c89e36b14..7b696cfcca 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 @@ -43,11 +43,11 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val conf = new SparkConf val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - val WORKER_TIMEOUT = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 - val RETAINED_APPLICATIONS = conf.getOrElse("spark.deploy.retainedApplications", "200").toInt - val REAPER_ITERATIONS = conf.getOrElse("spark.dead.worker.persistence", "15").toInt - val RECOVERY_DIR = conf.getOrElse("spark.deploy.recoveryDirectory", "") - val RECOVERY_MODE = conf.getOrElse("spark.deploy.recoveryMode", "NONE") + val WORKER_TIMEOUT = conf.get("spark.worker.timeout", "60").toLong * 1000 + val RETAINED_APPLICATIONS = conf.get("spark.deploy.retainedApplications", "200").toInt + val REAPER_ITERATIONS = conf.get("spark.dead.worker.persistence", "15").toInt + val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") + val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] @@ -88,7 +88,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - val spreadOutApps = conf.getOrElse("spark.deploy.spreadOut", "true").toBoolean + val spreadOutApps = conf.get("spark.deploy.spreadOut", "true").toBoolean override def preStart() { logInfo("Starting Spark master at " + masterUrl) 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 7ce83f9c36..e7f3224091 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 @@ -27,8 +27,8 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 7077 var webUiPort = 8080 - - // Check for settings in environment variables + + // Check for settings in environment variables if (System.getenv("SPARK_MASTER_HOST") != null) { host = System.getenv("SPARK_MASTER_HOST") } @@ -38,7 +38,7 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { if (System.getenv("SPARK_MASTER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_MASTER_WEBUI_PORT").toInt } - if (conf.get("master.ui.port") != null) { + if (conf.contains("master.ui.port")) { webUiPort = conf.get("master.ui.port").toInt } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala index 60c7a7c2d6..999090ad74 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala @@ -37,7 +37,7 @@ import org.apache.spark.{SparkConf, Logging} */ private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher, conf: SparkConf) extends Logging { - val ZK_URL = conf.getOrElse("spark.deploy.zookeeper.url", "") + val ZK_URL = conf.get("spark.deploy.zookeeper.url", "") val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE val ZK_TIMEOUT_MILLIS = 30000 diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index a61597bbdf..77c23fb9fb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -28,7 +28,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String, conf: SparkConf) extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging { - val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" + val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" private val watcher = new ZooKeeperWatcher() private val zk = new SparkZooKeeperSession(this, conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 245a558a59..52000d4f9c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -27,7 +27,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) with SparkZooKeeperWatcher with Logging { - val WORKING_DIR = conf.getOrElse("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" val zk = new SparkZooKeeperSession(this, conf) 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 f844fcbbfc..fcaf4e92b1 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 @@ -55,7 +55,7 @@ private[spark] class Worker( val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs // Send a heartbeat every (heartbeat timeout) / 4 milliseconds - val HEARTBEAT_MILLIS = conf.getOrElse("spark.worker.timeout", "60").toLong * 1000 / 4 + val HEARTBEAT_MILLIS = conf.get("spark.worker.timeout", "60").toLong * 1000 / 4 val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 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 a801d85770..c382034c99 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 @@ -37,7 +37,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val timeout = AkkaUtils.askTimeout(worker.conf) val host = Utils.localHostName() val port = requestedPort.getOrElse( - worker.conf.getOrElse("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) + worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) var server: Option[Server] = None var boundPort: Option[Int] = None 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 5b70165c35..3c92c205ea 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -302,7 +302,7 @@ private[spark] class Executor( * new classes defined by the REPL as the user types code */ private def addReplClassLoaderIfNeeded(parent: ClassLoader): ClassLoader = { - val classUri = conf.getOrElse("spark.repl.class.uri", null) + val classUri = conf.get("spark.repl.class.uri", null) if (classUri != null) { logInfo("Using REPL class URI: " + classUri) try { 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 075a18b068..a1e98845f6 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -39,7 +39,7 @@ trait CompressionCodec { private[spark] object CompressionCodec { def createCodec(conf: SparkConf): CompressionCodec = { - createCodec(conf, conf.getOrElse( + createCodec(conf, conf.get( "spark.io.compression.codec", classOf[LZFCompressionCodec].getName)) } @@ -71,7 +71,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = conf.getOrElse("spark.io.compression.snappy.block.size", "32768").toInt + val blockSize = conf.get("spark.io.compression.snappy.block.size", "32768").toInt new SnappyOutputStream(s, blockSize) } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 0e41c73ce7..9930537b34 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -65,7 +65,7 @@ import org.apache.spark.metrics.source.Source private[spark] class MetricsSystem private (val instance: String, conf: SparkConf) extends Logging { - val confFile = conf.getOrElse("spark.metrics.conf", null) + val confFile = conf.get("spark.metrics.conf", null) val metricsConfig = new MetricsConfig(Option(confFile)) val sinks = new mutable.ArrayBuffer[Sink] 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 697096fa76..46c40d0a2a 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -54,22 +54,22 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi private val selector = SelectorProvider.provider.openSelector() private val handleMessageExecutor = new ThreadPoolExecutor( - conf.getOrElse("spark.core.connection.handler.threads.min", "20").toInt, - conf.getOrElse("spark.core.connection.handler.threads.max", "60").toInt, - conf.getOrElse("spark.core.connection.handler.threads.keepalive", "60").toInt, TimeUnit.SECONDS, + conf.get("spark.core.connection.handler.threads.min", "20").toInt, + conf.get("spark.core.connection.handler.threads.max", "60").toInt, + conf.get("spark.core.connection.handler.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) private val handleReadWriteExecutor = new ThreadPoolExecutor( - conf.getOrElse("spark.core.connection.io.threads.min", "4").toInt, - conf.getOrElse("spark.core.connection.io.threads.max", "32").toInt, - conf.getOrElse("spark.core.connection.io.threads.keepalive", "60").toInt, TimeUnit.SECONDS, + conf.get("spark.core.connection.io.threads.min", "4").toInt, + conf.get("spark.core.connection.io.threads.max", "32").toInt, + conf.get("spark.core.connection.io.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : which should be executed asap private val handleConnectExecutor = new ThreadPoolExecutor( - conf.getOrElse("spark.core.connection.connect.threads.min", "1").toInt, - conf.getOrElse("spark.core.connection.connect.threads.max", "8").toInt, - conf.getOrElse("spark.core.connection.connect.threads.keepalive", "60").toInt, TimeUnit.SECONDS, + conf.get("spark.core.connection.connect.threads.min", "1").toInt, + conf.get("spark.core.connection.connect.threads.max", "8").toInt, + conf.get("spark.core.connection.connect.threads.keepalive", "60").toInt, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) private val serverChannel = ServerSocketChannel.open() diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala index db28ddf9ac..b729eb11c5 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala @@ -36,7 +36,7 @@ private[spark] class ShuffleCopier(conf: SparkConf) extends Logging { resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val connectTimeout = conf.getOrElse("spark.shuffle.netty.connect.timeout", "60000").toInt + val connectTimeout = conf.get("spark.shuffle.netty.connect.timeout", "60000").toInt val fc = new FileClient(handler, connectTimeout) try { diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 172ba6b01c..6d4f46125f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -97,7 +97,7 @@ private[spark] object CheckpointRDD extends Logging { throw new IOException("Checkpoint failed: temporary path " + tempOutputPath + " already exists") } - val bufferSize = env.conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = env.conf.get("spark.buffer.size", "65536").toInt val fileOutputStream = if (blockSize < 0) { fs.create(tempOutputPath, false, bufferSize) @@ -131,7 +131,7 @@ private[spark] object CheckpointRDD extends Logging { ): Iterator[T] = { val env = SparkEnv.get val fs = path.getFileSystem(broadcastedConf.value.value) - val bufferSize = env.conf.getOrElse("spark.buffer.size", "65536").toInt + val bufferSize = env.conf.get("spark.buffer.size", "65536").toInt val fileInputStream = fs.open(path, bufferSize) val serializer = env.serializer.newInstance() val deserializeStream = serializer.deserializeStream(fileInputStream) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 29b0247f8a..e22b1e53e8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.Utils private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedulerImpl) extends Logging { - private val THREADS = sparkEnv.conf.getOrElse("spark.resultGetter.threads", "4").toInt + private val THREADS = sparkEnv.conf.get("spark.resultGetter.threads", "4").toInt private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( THREADS, "Result resolver thread") diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index bffd990e16..d94b706854 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -51,15 +51,15 @@ private[spark] class TaskSchedulerImpl( isLocal: Boolean = false) extends TaskScheduler with Logging { - def this(sc: SparkContext) = this(sc, sc.conf.getOrElse("spark.task.maxFailures", "4").toInt) + def this(sc: SparkContext) = this(sc, sc.conf.get("spark.task.maxFailures", "4").toInt) val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL = conf.getOrElse("spark.speculation.interval", "100").toLong + val SPECULATION_INTERVAL = conf.get("spark.speculation.interval", "100").toLong // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT = conf.getOrElse("spark.starvation.timeout", "15000").toLong + val STARVATION_TIMEOUT = conf.get("spark.starvation.timeout", "15000").toLong // TaskSetManagers are not thread safe, so any access to one should be synchronized // on this class. @@ -96,7 +96,7 @@ private[spark] class TaskSchedulerImpl( var rootPool: Pool = null // default scheduler is FIFO val schedulingMode: SchedulingMode = SchedulingMode.withName( - conf.getOrElse("spark.scheduler.mode", "FIFO")) + conf.get("spark.scheduler.mode", "FIFO")) // This is a var so that we can reset it for testing purposes. private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) @@ -125,7 +125,7 @@ private[spark] class TaskSchedulerImpl( override def start() { backend.start() - if (!isLocal && conf.getOrElse("spark.speculation", "false").toBoolean) { + if (!isLocal && conf.get("spark.speculation", "false").toBoolean) { logInfo("Starting speculative execution thread") import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index b99664ae00..67ad99a4d7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -57,11 +57,11 @@ private[spark] class TaskSetManager( val conf = sched.sc.conf // CPUs to request per task - val CPUS_PER_TASK = conf.getOrElse("spark.task.cpus", "1").toInt + val CPUS_PER_TASK = conf.get("spark.task.cpus", "1").toInt // Quantile of tasks at which to start speculation - val SPECULATION_QUANTILE = conf.getOrElse("spark.speculation.quantile", "0.75").toDouble - val SPECULATION_MULTIPLIER = conf.getOrElse("spark.speculation.multiplier", "1.5").toDouble + val SPECULATION_QUANTILE = conf.get("spark.speculation.quantile", "0.75").toDouble + val SPECULATION_MULTIPLIER = conf.get("spark.speculation.multiplier", "1.5").toDouble // Serializer for closures and tasks. val env = SparkEnv.get @@ -116,7 +116,7 @@ private[spark] class TaskSetManager( // How frequently to reprint duplicate exceptions in full, in milliseconds val EXCEPTION_PRINT_INTERVAL = - conf.getOrElse("spark.logging.exceptionPrintInterval", "10000").toLong + conf.get("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 @@ -678,14 +678,14 @@ private[spark] class TaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = conf.getOrElse("spark.locality.wait", "3000") + val defaultWait = conf.get("spark.locality.wait", "3000") level match { case TaskLocality.PROCESS_LOCAL => - conf.getOrElse("spark.locality.wait.process", defaultWait).toLong + conf.get("spark.locality.wait.process", defaultWait).toLong case TaskLocality.NODE_LOCAL => - conf.getOrElse("spark.locality.wait.node", defaultWait).toLong + conf.get("spark.locality.wait.node", defaultWait).toLong case TaskLocality.RACK_LOCAL => - conf.getOrElse("spark.locality.wait.rack", defaultWait).toLong + conf.get("spark.locality.wait.rack", defaultWait).toLong case TaskLocality.ANY => 0L } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index b4a3ecca39..2f5bcafe40 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -22,7 +22,6 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ -import scala.util.Try import akka.actor._ import akka.pattern.ask @@ -64,7 +63,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) // Periodically revive offers to allow delay scheduling to work - val reviveInterval = conf.getOrElse("spark.scheduler.revive.interval", "1000").toLong + val reviveInterval = conf.get("spark.scheduler.revive.interval", "1000").toLong import context.dispatcher context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } @@ -209,8 +208,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A driverActor ! KillTask(taskId, executorId) } - override def defaultParallelism() = Try(conf.get("spark.default.parallelism")).toOption - .map(_.toInt).getOrElse(math.max(totalCoreCount.get(), 2)) + override def defaultParallelism(): Int = { + conf.getOption("spark.default.parallelism").map(_.toInt).getOrElse( + math.max(totalCoreCount.get(), 2)) + } // Called by subclasses when notified of a lost worker def removeExecutor(executorId: String, reason: String) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index f41fbbd1f3..b44d1e43c8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -33,7 +33,7 @@ private[spark] class SimrSchedulerBackend( val tmpPath = new Path(driverFilePath + "_tmp") val filePath = new Path(driverFilePath) - val maxCores = conf.getOrElse("spark.simr.executor.cores", "1").toInt + val maxCores = conf.get("spark.simr.executor.cores", "1").toInt override def start() { super.start() 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 224077566d..9858717d13 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 @@ -38,7 +38,7 @@ private[spark] class SparkDeploySchedulerBackend( var stopping = false var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ - val maxCores = conf.getOrElse("spark.cores.max", Int.MaxValue.toString).toInt + val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt override def start() { super.start() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 9e2cd3f699..d247fa4244 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -62,7 +62,7 @@ private[spark] class CoarseMesosSchedulerBackend( var driver: SchedulerDriver = null // Maximum number of cores to acquire (TODO: we'll need more flexible controls here) - val maxCores = conf.getOrElse("spark.cores.max", Int.MaxValue.toString).toInt + val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt // Cores we have acquired with each Mesos task ID val coresByTaskId = new HashMap[Int, Int] @@ -77,7 +77,7 @@ private[spark] class CoarseMesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) - val extraCoresPerSlave = conf.getOrElse("spark.mesos.extra.cores", "0").toInt + val extraCoresPerSlave = conf.get("spark.mesos.extra.cores", "0").toInt var nextMesosTaskId = 0 diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index be96382983..c20fc418e8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -340,5 +340,5 @@ private[spark] class MesosSchedulerBackend( } // TODO: query Mesos for number of cores - override def defaultParallelism() = sc.conf.getOrElse("spark.default.parallelism", "8").toInt + override def defaultParallelism() = sc.conf.get("spark.default.parallelism", "8").toInt } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 2367f3f521..a24a3b04b8 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -29,17 +29,14 @@ import org.apache.spark._ import org.apache.spark.broadcast.HttpBroadcast import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ -import scala.util.Try -import org.apache.spark.storage.PutBlock -import org.apache.spark.storage.GetBlock -import org.apache.spark.storage.GotBlock +import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. */ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer with Logging { private val bufferSize = { - conf.getOrElse("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 + conf.get("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 } def newKryoOutput() = new KryoOutput(bufferSize) @@ -51,7 +48,7 @@ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serial // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. // Do this before we invoke the user registrator so the user registrator can override this. - kryo.setReferences(conf.getOrElse("spark.kryo.referenceTracking", "true").toBoolean) + kryo.setReferences(conf.get("spark.kryo.referenceTracking", "true").toBoolean) for (cls <- KryoSerializer.toRegister) kryo.register(cls) @@ -61,13 +58,13 @@ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serial // Allow the user to register their own classes by setting spark.kryo.registrator try { - Try(conf.get("spark.kryo.registrator")).toOption.foreach { regCls => + for (regCls <- conf.getOption("spark.kryo.registrator")) { 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") + case e: Exception => logError("Failed to run spark.kryo.registrator", e) } // Register Chill's classes; we do this after our ranges and the user's own classes to let 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 3b25f68ca8..47478631a1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -327,7 +327,7 @@ object BlockFetcherIterator { fetchRequestsSync.put(request) } - copiers = startCopiers(conf.getOrElse("spark.shuffle.copier.threads", "6").toInt) + copiers = startCopiers(conf.get("spark.shuffle.copier.threads", "6").toInt) logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) 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 16ee208617..6d2cda97b0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -49,7 +49,7 @@ private[spark] class BlockManager( val shuffleBlockManager = new ShuffleBlockManager(this) val diskBlockManager = new DiskBlockManager(shuffleBlockManager, - conf.getOrElse("spark.local.dir", System.getProperty("java.io.tmpdir"))) + conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] @@ -58,8 +58,8 @@ private[spark] class BlockManager( // If we use Netty for shuffle, start a new Netty-based shuffle sender service. private val nettyPort: Int = { - val useNetty = conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean - val nettyPortConfig = conf.getOrElse("spark.shuffle.sender.port", "0").toInt + val useNetty = conf.get("spark.shuffle.use.netty", "false").toBoolean + val nettyPortConfig = conf.get("spark.shuffle.sender.port", "0").toInt if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 } @@ -72,14 +72,14 @@ private[spark] class BlockManager( // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory // for receiving shuffle outputs) val maxBytesInFlight = - conf.getOrElse("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 + conf.get("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024 // Whether to compress broadcast variables that are stored - val compressBroadcast = conf.getOrElse("spark.broadcast.compress", "true").toBoolean + val compressBroadcast = conf.get("spark.broadcast.compress", "true").toBoolean // Whether to compress shuffle output that are stored - val compressShuffle = conf.getOrElse("spark.shuffle.compress", "true").toBoolean + val compressShuffle = conf.get("spark.shuffle.compress", "true").toBoolean // Whether to compress RDD partitions that are stored serialized - val compressRdds = conf.getOrElse("spark.rdd.compress", "false").toBoolean + val compressRdds = conf.get("spark.rdd.compress", "false").toBoolean val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf) @@ -443,7 +443,7 @@ private[spark] class BlockManager( : BlockFetcherIterator = { val iter = - if (conf.getOrElse("spark.shuffle.use.netty", "false").toBoolean) { + if (conf.get("spark.shuffle.use.netty", "false").toBoolean) { new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) } else { new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer) @@ -469,7 +469,7 @@ private[spark] class BlockManager( def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int) : BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) - val syncWrites = conf.getOrElse("spark.shuffle.sync", "false").toBoolean + val syncWrites = conf.get("spark.shuffle.sync", "false").toBoolean new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites) } @@ -864,15 +864,15 @@ private[spark] object BlockManager extends Logging { val ID_GENERATOR = new IdGenerator def getMaxMemory(conf: SparkConf): Long = { - val memoryFraction = conf.getOrElse("spark.storage.memoryFraction", "0.66").toDouble + val memoryFraction = conf.get("spark.storage.memoryFraction", "0.66").toDouble (Runtime.getRuntime.maxMemory * memoryFraction).toLong } def getHeartBeatFrequency(conf: SparkConf): Long = - conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 + conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 def getDisableHeartBeatsForTesting(conf: SparkConf): Boolean = - conf.getOrElse("spark.test.disableBlockManagerHeartBeat", "false").toBoolean + conf.get("spark.test.disableBlockManagerHeartBeat", "false").toBoolean /** * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 8e4a88b20a..b5afe8cd23 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -31,8 +31,8 @@ private[spark] class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection], conf: SparkConf) extends Logging { - val AKKA_RETRY_ATTEMPTS: Int = conf.getOrElse("spark.akka.num.retries", "3").toInt - val AKKA_RETRY_INTERVAL_MS: Int = conf.getOrElse("spark.akka.retry.wait", "3000").toInt + val AKKA_RETRY_ATTEMPTS: Int = conf.get("spark.akka.num.retries", "3").toInt + val AKKA_RETRY_INTERVAL_MS: Int = conf.get("spark.akka.retry.wait", "3000").toInt val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" 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 dbbeeb39eb..58452d9657 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -50,10 +50,10 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act private val akkaTimeout = AkkaUtils.askTimeout(conf) - val slaveTimeout = conf.getOrElse("spark.storage.blockManagerSlaveTimeoutMs", + val slaveTimeout = conf.get("spark.storage.blockManagerSlaveTimeoutMs", "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong - val checkTimeoutInterval = conf.getOrElse("spark.storage.blockManagerTimeoutIntervalMs", + val checkTimeoutInterval = conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong var timeoutCheckingTask: Cancellable = null diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 7697092e1b..55dcb3742c 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -38,7 +38,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD extends PathResolver with Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 - private val subDirsPerLocalDir = shuffleManager.conf.getOrElse("spark.diskStore.subDirectories", "64").toInt + private val subDirsPerLocalDir = shuffleManager.conf.get("spark.diskStore.subDirectories", "64").toInt // Create one local directory for each path mentioned in spark.local.dir; then, inside this // directory, create multiple subdirectories that we will hash files into, in order to avoid diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 151eedb783..39dc7bb19a 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -27,8 +27,6 @@ import org.apache.spark.serializer.Serializer import org.apache.spark.util.{MetadataCleanerType, MetadataCleaner, TimeStampedHashMap} import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} import org.apache.spark.storage.ShuffleBlockManager.ShuffleFileGroup -import scala.util.Try -import org.apache.spark.SparkConf /** A group of writers for a ShuffleMapTask, one writer per reducer. */ private[spark] trait ShuffleWriterGroup { @@ -66,9 +64,9 @@ class ShuffleBlockManager(blockManager: BlockManager) { // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. // TODO: Remove this once the shuffle file consolidation feature is stable. val consolidateShuffleFiles = - conf.getOrElse("spark.shuffle.consolidateFiles", "false").toBoolean + conf.get("spark.shuffle.consolidateFiles", "false").toBoolean - private val bufferSize = conf.getOrElse("spark.shuffle.file.buffer.kb", "100").toInt * 1024 + private val bufferSize = conf.get("spark.shuffle.file.buffer.kb", "100").toInt * 1024 /** * Contains all the state related to a particular shuffle. This includes a pool of unused 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 0ce8d9c8c4..50dfdbdf5a 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -1,4 +1,4 @@ -/* +/* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) - val port = sc.conf.getOrElse("spark.ui.port", SparkUI.DEFAULT_PORT).toInt + val port = sc.conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt var boundPort: Option[Int] = None var server: Option[Server] = None diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 6b4602f928..88f41be8d3 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -66,7 +66,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) { UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties, fixedWidth = true) val classPathEntries = classPathProperty._2 - .split(sc.conf.getOrElse("path.separator", ":")) + .split(sc.conf.get("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")} 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 315014d27d..b7b87250b9 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 @@ -33,7 +33,7 @@ import org.apache.spark.scheduler._ */ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { // How many stages to remember - val RETAINED_STAGES = sc.conf.getOrElse("spark.ui.retained_stages", "1000").toInt + val RETAINED_STAGES = sc.conf.get("spark.ui.retained_stages", "1000").toInt val DEFAULT_POOL_NAME = "default" val stageIdToPool = new HashMap[Int, String]() diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 58b26f7f12..362cea5e3e 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -41,19 +41,19 @@ private[spark] object AkkaUtils { def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false, conf: SparkConf): (ActorSystem, Int) = { - val akkaThreads = conf.getOrElse("spark.akka.threads", "4").toInt - val akkaBatchSize = conf.getOrElse("spark.akka.batchSize", "15").toInt + val akkaThreads = conf.get("spark.akka.threads", "4").toInt + val akkaBatchSize = conf.get("spark.akka.batchSize", "15").toInt - val akkaTimeout = conf.getOrElse("spark.akka.timeout", "100").toInt + val akkaTimeout = conf.get("spark.akka.timeout", "100").toInt - val akkaFrameSize = conf.getOrElse("spark.akka.frameSize", "10").toInt + val akkaFrameSize = conf.get("spark.akka.frameSize", "10").toInt val lifecycleEvents = - if (conf.getOrElse("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" + if (conf.get("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - val akkaHeartBeatPauses = conf.getOrElse("spark.akka.heartbeat.pauses", "600").toInt + val akkaHeartBeatPauses = conf.get("spark.akka.heartbeat.pauses", "600").toInt val akkaFailureDetector = - conf.getOrElse("spark.akka.failure-detector.threshold", "300.0").toDouble - val akkaHeartBeatInterval = conf.getOrElse("spark.akka.heartbeat.interval", "1000").toInt + conf.get("spark.akka.failure-detector.threshold", "300.0").toDouble + val akkaHeartBeatInterval = conf.get("spark.akka.heartbeat.interval", "1000").toInt val akkaConf = ConfigFactory.parseString( s""" @@ -89,6 +89,6 @@ private[spark] object AkkaUtils { /** Returns the default Spark timeout to use for Akka ask operations. */ def askTimeout(conf: SparkConf): FiniteDuration = { - Duration.create(conf.getOrElse("spark.akka.askTimeout", "30").toLong, "seconds") + Duration.create(conf.get("spark.akka.askTimeout", "30").toLong, "seconds") } } diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 9ea7fc2dfd..aa7f52cafb 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -74,12 +74,12 @@ object MetadataCleanerType extends Enumeration { // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. object MetadataCleaner { def getDelaySeconds(conf: SparkConf) = { - conf.getOrElse("spark.cleaner.ttl", "3500").toInt + conf.get("spark.cleaner.ttl", "3500").toInt } def getDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType): Int = { - conf.getOrElse(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString) + conf.get(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString) .toInt } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index ca3320b22b..5f1253100b 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -311,7 +311,7 @@ private[spark] object Utils extends Logging { * multiple paths. */ def getLocalDir(conf: SparkConf): String = { - conf.getOrElse("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) + conf.get("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) } /** @@ -397,7 +397,7 @@ private[spark] object Utils extends Logging { } def localHostPort(conf: SparkConf): String = { - val retval = conf.getOrElse("spark.hostPort", null) + val retval = conf.get("spark.hostPort", null) if (retval == null) { logErrorWithStack("spark.hostPort not set but invoking localHostPort") return localHostName() diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 77c7b829b3..ef5936dd2f 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -74,7 +74,7 @@ class SparkConfSuite extends FunSuite with LocalSparkContext { assert(!conf.contains("k4"), "conf contained k4") assert(conf.get("k1") === "v4") intercept[Exception] { conf.get("k4") } - assert(conf.getOrElse("k4", "not found") === "not found") + assert(conf.get("k4", "not found") === "not found") assert(conf.getOption("k1") === Some("v4")) assert(conf.getOption("k4") === None) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 5d33e66253..1eec6726f4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -83,7 +83,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { private val conf = new SparkConf - val LOCALITY_WAIT = conf.getOrElse("spark.locality.wait", "3000").toLong + val LOCALITY_WAIT = conf.get("spark.locality.wait", "3000").toLong val MAX_TASK_FAILURES = 4 test("TaskSet with no preferences") { diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index f940448abd..af4b31d53c 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -27,8 +27,8 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import scala.util.Try import akka.actor.{Props, ActorSelection, ActorSystem} -class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { - private val testConf = new SparkConf +class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { + private val testConf = new SparkConf(false) val rootDir0 = Files.createTempDir() rootDir0.deleteOnExit() val rootDir1 = Files.createTempDir() @@ -38,9 +38,7 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before // This suite focuses primarily on consolidation features, // so we coerce consolidation if not already enabled. - val consolidateProp = "spark.shuffle.consolidateFiles" - val oldConsolidate = Try(testConf.get(consolidateProp)).toOption - testConf.set(consolidateProp, "true") + testConf.set("spark.shuffle.consolidateFiles", "true") val shuffleBlockManager = new ShuffleBlockManager(null) { override def conf = testConf.clone @@ -50,10 +48,6 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before var diskBlockManager: DiskBlockManager = _ - override def afterAll() { - oldConsolidate.map(c => System.setProperty(consolidateProp, c)) - } - override def beforeEach() { diskBlockManager = new DiskBlockManager(shuffleBlockManager, rootDirs) shuffleBlockManager.idToSegmentMap.clear() diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index 9dcdcfaa67..c111e2e90f 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -93,7 +93,7 @@ class SparkConf(object): def set(self, key, value): """Set a configuration property.""" - self._jconf.set(key, value) + self._jconf.set(key, unicode(value)) return self def setMaster(self, value): @@ -132,13 +132,9 @@ class SparkConf(object): self._jconf.set(k, v) return self - def get(self, key): - """Get the configured value for some key, if set.""" - return self._jconf.get(key) - - def getOrElse(self, key, defaultValue): - """Get the value for some key, or return a default otherwise.""" - return self._jconf.getOrElse(key, defaultValue) + def get(self, key, defaultValue=None): + """Get the configured value for some key, or return a default otherwise.""" + return self._jconf.get(key, defaultValue) def getAll(self): """Get all values as a list of key-value pairs.""" 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 a993083164..59fdb0b37a 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -89,7 +89,7 @@ import org.apache.spark.util.Utils /** Local directory to save .class files too */ val outputDir = { val tmp = System.getProperty("java.io.tmpdir") - val rootDir = new SparkConf().getOrElse("spark.repl.classdir", tmp) + val rootDir = new SparkConf().get("spark.repl.classdir", tmp) Utils.createTempDir(rootDir) } if (SPARK_DEBUG_REPL) { 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 a230845b92..27d474c0a0 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 @@ -174,8 +174,8 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging /** A helper actor that communicates with the NetworkInputTracker */ private class NetworkReceiverActor extends Actor { logInfo("Attempting to register with tracker") - val ip = env.conf.getOrElse("spark.driver.host", "localhost") - val port = env.conf.getOrElse("spark.driver.port", "7077").toInt + val ip = env.conf.get("spark.driver.host", "localhost") + val port = env.conf.get("spark.driver.port", "7077").toInt val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) val tracker = env.actorSystem.actorSelection(url) val timeout = 5.seconds @@ -212,7 +212,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null) val clock = new SystemClock() - val blockInterval = env.conf.getOrElse("spark.streaming.blockInterval", "200").toLong + val blockInterval = env.conf.get("spark.streaming.blockInterval", "200").toLong val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer) val blockStorageLevel = storageLevel val blocksForPushing = new ArrayBlockingQueue[Block](1000) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 844180c81a..5f8be93a98 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -46,7 +46,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { } })) val clock = { - val clockClass = ssc.sc.conf.getOrElse( + val clockClass = ssc.sc.conf.get( "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") Class.forName(clockClass).newInstance().asInstanceOf[Clock] } @@ -104,7 +104,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { // or if the property is defined set it to that time if (clock.isInstanceOf[ManualClock]) { val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds - val jumpTime = ssc.sc.conf.getOrElse("spark.streaming.manualClock.jump", "0").toLong + val jumpTime = ssc.sc.conf.get("spark.streaming.manualClock.jump", "0").toLong clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 651cdaaa6d..9304fc1a93 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -31,7 +31,7 @@ private[streaming] class JobScheduler(val ssc: StreamingContext) extends Logging { val jobSets = new ConcurrentHashMap[Time, JobSet] - val numConcurrentJobs = ssc.conf.getOrElse("spark.streaming.concurrentJobs", "1").toInt + val numConcurrentJobs = ssc.conf.get("spark.streaming.concurrentJobs", "1").toInt val executor = Executors.newFixedThreadPool(numConcurrentJobs) val generator = new JobGenerator(this) val listenerBus = new StreamingListenerBus() -- cgit v1.2.3 From 0f6060733da83a862038fd397875cdb49d8c144d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 1 Jan 2014 22:09:42 -0500 Subject: Fixed two uses of conf.get with no default value in Mesos --- .../spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index d247fa4244..08811520cf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -125,7 +125,7 @@ private[spark] class CoarseMesosSchedulerBackend( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val uri = conf.get("spark.executor.uri") + val uri = conf.get("spark.executor.uri", null) if (uri == null) { val runScript = new File(sparkHome, "spark-class").getCanonicalPath command.setValue( diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index c20fc418e8..bb278fb155 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -100,7 +100,7 @@ private[spark] class MesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val uri = sc.conf.get("spark.executor.uri") + val uri = sc.conf.get("spark.executor.uri", null) if (uri == null) { command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath) } else { -- cgit v1.2.3 From 436f3d28560bed9f428efce6f7c4caf44111c60e Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 2 Jan 2014 16:08:35 +0530 Subject: ignoring tests for now, contrary to what I assumed these tests make sense given what they are testing. --- core/src/test/resources/uncommons-maths-1.2.2.jar | Bin 49019 -> 0 bytes .../test/scala/org/apache/spark/FileServerSuite.scala | 6 +++--- 2 files changed, 3 insertions(+), 3 deletions(-) delete mode 100644 core/src/test/resources/uncommons-maths-1.2.2.jar (limited to 'core/src') diff --git a/core/src/test/resources/uncommons-maths-1.2.2.jar b/core/src/test/resources/uncommons-maths-1.2.2.jar deleted file mode 100644 index e126001c1c..0000000000 Binary files a/core/src/test/resources/uncommons-maths-1.2.2.jar and /dev/null differ diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index c210dd5c3b..063b5fbab4 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -75,7 +75,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { assert(result.toSet === Set((1,200), (2,300), (3,500))) } - test ("Dynamically adding JARS locally") { + ignore ("Dynamically adding JARS locally") { sc = new SparkContext("local[4]", "test") val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile() sc.addJar(sampleJarFile) @@ -105,7 +105,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { assert(result.toSet === Set((1,200), (2,300), (3,500))) } - test ("Dynamically adding JARS on a standalone cluster") { + ignore ("Dynamically adding JARS on a standalone cluster") { sc = new SparkContext("local-cluster[1,1,512]", "test") val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile() sc.addJar(sampleJarFile) @@ -121,7 +121,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { assert(result.toSet === Set((1,2), (2,7), (3,121))) } - test ("Dynamically adding JARS on a standalone cluster using local: URL") { + ignore ("Dynamically adding JARS on a standalone cluster using local: URL") { sc = new SparkContext("local-cluster[1,1,512]", "test") val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile() sc.addJar(sampleJarFile.replace("file", "local")) -- cgit v1.2.3 From 08ec10de1767ca543047b79c40ab50a04ce5df2f Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 2 Jan 2014 17:31:33 +0530 Subject: Removed a repeated test and changed tests to not use uncommons jar --- .../scala/org/apache/spark/FileServerSuite.scala | 94 ++++++++++++---------- 1 file changed, 50 insertions(+), 44 deletions(-) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 063b5fbab4..a15c3751c2 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -17,15 +17,46 @@ package org.apache.spark +import java.io._ +import java.util.jar.{JarEntry, JarOutputStream} + +import SparkContext._ import com.google.common.io.Files import org.scalatest.FunSuite -import java.io.{File, PrintWriter, FileReader, BufferedReader} -import SparkContext._ class FileServerSuite extends FunSuite with LocalSparkContext { @transient var tmpFile: File = _ - @transient var testJarFile: File = _ + @transient var testJarFile: String = _ + + + override def beforeAll() { + super.beforeAll() + val buffer = new Array[Byte](10240) + val tmpdir = new File(Files.createTempDir(), "test") + tmpdir.mkdir() + val tmpJarEntry = new File(tmpdir, "FileServerSuite2.txt") + val pw = new PrintWriter(tmpJarEntry) + pw.println("test String in the file named FileServerSuite2.txt") + pw.close() + // The ugliest code possible, was translated from java. + val tmpFile2 = new File(tmpdir, "test.jar") + val stream = new FileOutputStream(tmpFile2) + val jar = new JarOutputStream(stream, new java.util.jar.Manifest()) + val jarAdd = new JarEntry(tmpJarEntry.getName) + jarAdd.setTime(tmpJarEntry.lastModified) + jar.putNextEntry(jarAdd) + val in = new FileInputStream(tmpJarEntry) + var nRead = 0 + while (nRead <= 0) { + nRead = in.read(buffer, 0, buffer.length) + jar.write(buffer, 0, nRead) + } + in.close() + jar.close() + stream.close() + testJarFile = tmpFile2.getAbsolutePath + } override def beforeEach() { super.beforeEach() @@ -75,20 +106,15 @@ class FileServerSuite extends FunSuite with LocalSparkContext { assert(result.toSet === Set((1,200), (2,300), (3,500))) } - ignore ("Dynamically adding JARS locally") { + test ("Dynamically adding JARS locally") { sc = new SparkContext("local[4]", "test") - val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile() - sc.addJar(sampleJarFile) - val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0)) - val result = sc.parallelize(testData).reduceByKey { (x,y) => - val fac = Thread.currentThread.getContextClassLoader() - .loadClass("org.uncommons.maths.Maths") - .getDeclaredMethod("factorial", classOf[Int]) - val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt - val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt - a + b - }.collect() - assert(result.toSet === Set((1,2), (2,7), (3,121))) + sc.addJar(testJarFile) + val testData = Array((1, 1)) + sc.parallelize(testData).foreach { (x) => + if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite2.txt") == null) { + throw new SparkException("jar not added") + } + } } test("Distributing files on a standalone cluster") { @@ -105,35 +131,15 @@ class FileServerSuite extends FunSuite with LocalSparkContext { assert(result.toSet === Set((1,200), (2,300), (3,500))) } - ignore ("Dynamically adding JARS on a standalone cluster") { + test ("Dynamically adding JARS on a standalone cluster") { sc = new SparkContext("local-cluster[1,1,512]", "test") - val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile() - sc.addJar(sampleJarFile) - val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0)) - val result = sc.parallelize(testData).reduceByKey { (x,y) => - val fac = Thread.currentThread.getContextClassLoader() - .loadClass("org.uncommons.maths.Maths") - .getDeclaredMethod("factorial", classOf[Int]) - val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt - val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt - a + b - }.collect() - assert(result.toSet === Set((1,2), (2,7), (3,121))) + sc.addJar(testJarFile) + val testData = Array((1,1)) + sc.parallelize(testData).foreach { (x) => + if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite2.txt") == null) { + throw new SparkException("jar not added") + } + } } - ignore ("Dynamically adding JARS on a standalone cluster using local: URL") { - sc = new SparkContext("local-cluster[1,1,512]", "test") - val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile() - sc.addJar(sampleJarFile.replace("file", "local")) - val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0)) - val result = sc.parallelize(testData).reduceByKey { (x,y) => - val fac = Thread.currentThread.getContextClassLoader() - .loadClass("org.uncommons.maths.Maths") - .getDeclaredMethod("factorial", classOf[Int]) - val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt - val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt - a + b - }.collect() - assert(result.toSet === Set((1,2), (2,7), (3,121))) - } } -- cgit v1.2.3 From 66d501276b5a066bd9abaa4e284cfad557665948 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 2 Jan 2014 16:17:57 +0000 Subject: Suggested small changes to Java code for slightly more standard style, encapsulation and in some cases performance --- .../org/apache/spark/network/netty/FileClient.java | 5 ++- .../netty/FileClientChannelInitializer.java | 2 +- .../org/apache/spark/network/netty/FileServer.java | 8 ++-- .../netty/FileServerChannelInitializer.java | 4 +- .../spark/network/netty/FileServerHandler.java | 6 +-- .../apache/spark/network/netty/PathResolver.java | 52 +++++++++++----------- .../java/org/apache/spark/examples/JavaHdfsLR.java | 32 +++++++------ .../java/org/apache/spark/examples/JavaKMeans.java | 26 +++++++---- .../org/apache/spark/examples/JavaLogQuery.java | 23 +++++----- .../org/apache/spark/examples/JavaPageRank.java | 14 ++++-- .../org/apache/spark/examples/JavaSparkPi.java | 11 +++-- .../java/org/apache/spark/examples/JavaTC.java | 19 +++++--- .../org/apache/spark/examples/JavaWordCount.java | 13 +++++- .../org/apache/spark/mllib/examples/JavaALS.java | 21 ++++++--- .../apache/spark/mllib/examples/JavaKMeans.java | 19 +++++--- .../org/apache/spark/mllib/examples/JavaLR.java | 22 +++++---- .../streaming/examples/JavaFlumeEventCount.java | 5 ++- .../streaming/examples/JavaKafkaWordCount.java | 16 ++++--- .../streaming/examples/JavaNetworkWordCount.java | 15 +++++-- .../spark/streaming/examples/JavaQueueStream.java | 11 +++-- 20 files changed, 203 insertions(+), 121 deletions(-) (limited to 'core/src') diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java index 46d61503bc..d2d778b756 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClient.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClient.java @@ -31,7 +31,8 @@ import java.util.concurrent.TimeUnit; class FileClient { - private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); + private static final Logger LOG = LoggerFactory.getLogger(FileClient.class.getName()); + private final FileClientHandler handler; private Channel channel = null; private Bootstrap bootstrap = null; @@ -39,7 +40,7 @@ class FileClient { private final int connectTimeout; private final int sendTimeout = 60; // 1 min - public FileClient(FileClientHandler handler, int connectTimeout) { + FileClient(FileClientHandler handler, int connectTimeout) { this.handler = handler; this.connectTimeout = connectTimeout; } diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java index fb61be1c12..264cf97d02 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java @@ -25,7 +25,7 @@ class FileClientChannelInitializer extends ChannelInitializer { private final FileClientHandler fhandler; - public FileClientChannelInitializer(FileClientHandler handler) { + FileClientChannelInitializer(FileClientHandler handler) { fhandler = handler; } diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServer.java b/core/src/main/java/org/apache/spark/network/netty/FileServer.java index aea7534459..c93425e278 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServer.java @@ -33,15 +33,14 @@ import org.slf4j.LoggerFactory; */ class FileServer { - private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); + private static final Logger LOG = LoggerFactory.getLogger(FileServer.class.getName()); private EventLoopGroup bossGroup = null; private EventLoopGroup workerGroup = null; private ChannelFuture channelFuture = null; private int port = 0; - private Thread blockingThread = null; - public FileServer(PathResolver pResolver, int port) { + FileServer(PathResolver pResolver, int port) { InetSocketAddress addr = new InetSocketAddress(port); // Configure the server. @@ -70,7 +69,8 @@ class FileServer { * Start the file server asynchronously in a new thread. */ public void start() { - blockingThread = new Thread() { + Thread blockingThread = new Thread() { + @Override public void run() { try { channelFuture.channel().closeFuture().sync(); diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java index 3f15ff898f..46efec8f8d 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java @@ -25,9 +25,9 @@ import io.netty.handler.codec.string.StringDecoder; class FileServerChannelInitializer extends ChannelInitializer { - PathResolver pResolver; + private final PathResolver pResolver; - public FileServerChannelInitializer(PathResolver pResolver) { + FileServerChannelInitializer(PathResolver pResolver) { this.pResolver = pResolver; } diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java index e2d9391b4c..3ac045f944 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java @@ -31,11 +31,11 @@ import org.slf4j.LoggerFactory; class FileServerHandler extends SimpleChannelInboundHandler { - private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); + private static final Logger LOG = LoggerFactory.getLogger(FileServerHandler.class.getName()); private final PathResolver pResolver; - public FileServerHandler(PathResolver pResolver){ + FileServerHandler(PathResolver pResolver){ this.pResolver = pResolver; } @@ -61,7 +61,7 @@ class FileServerHandler extends SimpleChannelInboundHandler { ctx.flush(); return; } - int len = new Long(length).intValue(); + int len = (int) length; ctx.write((new FileHeader(len, blockId)).buffer()); try { ctx.write(new DefaultFileRegion(new FileInputStream(file) diff --git a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java index 9f7ced44cf..7ad8d03efb 100755 --- a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java +++ b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java @@ -1,26 +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 org.apache.spark.network.netty; - -import org.apache.spark.storage.BlockId; -import org.apache.spark.storage.FileSegment; - -public interface PathResolver { - /** Get the file segment in which the given block resides. */ - public FileSegment getBlockLocation(BlockId blockId); -} +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.network.netty; + +import org.apache.spark.storage.BlockId; +import org.apache.spark.storage.FileSegment; + +public interface PathResolver { + /** Get the file segment in which the given block resides. */ + FileSegment getBlockLocation(BlockId blockId); +} diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index be0d38589c..9f0e3412a6 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -24,19 +24,22 @@ import org.apache.spark.api.java.function.Function2; import java.io.Serializable; import java.util.Arrays; -import java.util.StringTokenizer; import java.util.Random; +import java.util.regex.Pattern; /** * Logistic regression based classification. */ -public class JavaHdfsLR { +public final class JavaHdfsLR { - static int D = 10; // Number of dimensions - static Random rand = new Random(42); + private static final int D = 10; // Number of dimensions + private static final Random rand = new Random(42); + + private JavaHdfsLR() { + } static class DataPoint implements Serializable { - public DataPoint(double[] x, double y) { + DataPoint(double[] x, double y) { this.x = x; this.y = y; } @@ -46,20 +49,22 @@ public class JavaHdfsLR { } static class ParsePoint extends Function { + private static final Pattern SPACE = Pattern.compile(" "); + + @Override public DataPoint call(String line) { - StringTokenizer tok = new StringTokenizer(line, " "); - double y = Double.parseDouble(tok.nextToken()); + String[] tok = SPACE.split(line); + double y = Double.parseDouble(tok[0]); double[] x = new double[D]; - int i = 0; - while (i < D) { - x[i] = Double.parseDouble(tok.nextToken()); - i += 1; + for (int i = 0; i < D; i++) { + x[i] = Double.parseDouble(tok[i+1]); } return new DataPoint(x, y); } } static class VectorSum extends Function2 { + @Override public double[] call(double[] a, double[] b) { double[] result = new double[D]; for (int j = 0; j < D; j++) { @@ -70,12 +75,13 @@ public class JavaHdfsLR { } static class ComputeGradient extends Function { - double[] weights; + private final double[] weights; - public ComputeGradient(double[] weights) { + ComputeGradient(double[] weights) { this.weights = weights; } + @Override public double[] call(DataPoint p) { double[] gradient = new double[D]; for (int i = 0; i < D; i++) { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java index 5a6afe7eae..1671d0cdc8 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java @@ -27,19 +27,27 @@ import org.apache.spark.util.Vector; import java.util.List; import java.util.Map; +import java.util.regex.Pattern; /** * K-means clustering using Java API. */ -public class JavaKMeans { +public final class JavaKMeans { + + private static final Pattern SPACE = Pattern.compile(" "); + + private JavaKMeans() { + } /** Parses numbers split by whitespace to a vector */ static Vector parseVector(String line) { - String[] splits = line.split(" "); + String[] splits = SPACE.split(line); double[] data = new double[splits.length]; int i = 0; - for (String s : splits) - data[i] = Double.parseDouble(splits[i++]); + for (String s : splits) { + data[i] = Double.parseDouble(s); + i++; + } return new Vector(data); } @@ -82,7 +90,7 @@ public class JavaKMeans { JavaRDD data = sc.textFile(path).map( new Function() { @Override - public Vector call(String line) throws Exception { + public Vector call(String line) { return parseVector(line); } } @@ -96,7 +104,7 @@ public class JavaKMeans { JavaPairRDD closest = data.map( new PairFunction() { @Override - public Tuple2 call(Vector vector) throws Exception { + public Tuple2 call(Vector vector) { return new Tuple2( closestPoint(vector, centroids), vector); } @@ -107,7 +115,8 @@ public class JavaKMeans { JavaPairRDD> pointsGroup = closest.groupByKey(); Map newCentroids = pointsGroup.mapValues( new Function, Vector>() { - public Vector call(List ps) throws Exception { + @Override + public Vector call(List ps) { return average(ps); } }).collectAsMap(); @@ -122,8 +131,9 @@ public class JavaKMeans { } while (tempDist > convergeDist); System.out.println("Final centers:"); - for (Vector c : centroids) + for (Vector c : centroids) { System.out.println(c); + } System.exit(0); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index 407cd7ccfa..1ce53fe403 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -35,9 +35,9 @@ import java.util.regex.Pattern; /** * Executes a roll up-style query against Apache logs. */ -public class JavaLogQuery { +public final class JavaLogQuery { - public static List exampleApacheLogs = Lists.newArrayList( + public static final List exampleApacheLogs = Lists.newArrayList( "10.10.10.10 - \"FRED\" [18/Jan/2013:17:56:07 +1100] \"GET http://images.com/2013/Generic.jpg " + "HTTP/1.1\" 304 315 \"http://referall.com/\" \"Mozilla/4.0 (compatible; MSIE 7.0; " + "Windows NT 5.1; GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; " + @@ -51,14 +51,17 @@ public class JavaLogQuery { "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.352 \"-\" - \"\" 256 977 988 \"\" " + "0 73.23.2.15 images.com 1358492557 - Whatup"); - public static Pattern apacheLogRegex = Pattern.compile( + public static final Pattern apacheLogRegex = Pattern.compile( "^([\\d.]+) (\\S+) (\\S+) \\[([\\w\\d:/]+\\s[+\\-]\\d{4})\\] \"(.+?)\" (\\d{3}) ([\\d\\-]+) \"([^\"]+)\" \"([^\"]+)\".*"); + private JavaLogQuery() { + } + /** Tracks the total query count and number of aggregate bytes for a particular group. */ public static class Stats implements Serializable { - private int count; - private int numBytes; + private final int count; + private final int numBytes; public Stats(int count, int numBytes) { this.count = count; @@ -92,12 +95,12 @@ public class JavaLogQuery { if (m.find()) { int bytes = Integer.parseInt(m.group(7)); return new Stats(1, bytes); - } - else + } else { return new Stats(1, 0); + } } - public static void main(String[] args) throws Exception { + public static void main(String[] args) { if (args.length == 0) { System.err.println("Usage: JavaLogQuery [logFile]"); System.exit(1); @@ -110,14 +113,14 @@ public class JavaLogQuery { JavaPairRDD, Stats> extracted = dataSet.map(new PairFunction, Stats>() { @Override - public Tuple2, Stats> call(String s) throws Exception { + public Tuple2, Stats> call(String s) { return new Tuple2, Stats>(extractKey(s), extractStats(s)); } }); JavaPairRDD, Stats> counts = extracted.reduceByKey(new Function2() { @Override - public Stats call(Stats stats, Stats stats2) throws Exception { + public Stats call(Stats stats, Stats stats2) { return stats.merge(stats2); } }); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index 89aed8f279..447ba93bd6 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -28,6 +28,7 @@ import org.apache.spark.api.java.function.PairFunction; import java.util.List; import java.util.ArrayList; +import java.util.regex.Pattern; /** * Computes the PageRank of URLs from an input file. Input file should @@ -38,7 +39,12 @@ import java.util.ArrayList; * ... * where URL and their neighbors are separated by space(s). */ -public class JavaPageRank { +public final class JavaPageRank { + private static final Pattern SPACES = Pattern.compile("\\s+"); + + private JavaPageRank() { + } + private static class Sum extends Function2 { @Override public Double call(Double a, Double b) { @@ -66,7 +72,7 @@ public class JavaPageRank { JavaPairRDD> links = lines.map(new PairFunction() { @Override public Tuple2 call(String s) { - String[] parts = s.split("\\s+"); + String[] parts = SPACES.split(s); return new Tuple2(parts[0], parts[1]); } }).distinct().groupByKey().cache(); @@ -74,7 +80,7 @@ public class JavaPageRank { // Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. JavaPairRDD ranks = links.mapValues(new Function, Double>() { @Override - public Double call(List rs) throws Exception { + public Double call(List rs) { return 1.0; } }); @@ -97,7 +103,7 @@ public class JavaPageRank { // Re-calculates URL ranks based on neighbor contributions. ranks = contribs.reduceByKey(new Sum()).mapValues(new Function() { @Override - public Double call(Double sum) throws Exception { + public Double call(Double sum) { return 0.15 + sum * 0.85; } }); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index 4a2380caf5..d2a2a1db7c 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -26,8 +26,10 @@ import java.util.ArrayList; import java.util.List; /** Computes an approximation to pi */ -public class JavaSparkPi { +public final class JavaSparkPi { + private JavaSparkPi() { + } public static void main(String[] args) throws Exception { if (args.length == 0) { @@ -41,21 +43,22 @@ public class JavaSparkPi { int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2; int n = 100000 * slices; List l = new ArrayList(n); - for (int i = 0; i < n; i++) + for (int i = 0; i < n; i++) { l.add(i); + } JavaRDD dataSet = jsc.parallelize(l, slices); int count = dataSet.map(new Function() { @Override - public Integer call(Integer integer) throws Exception { + public Integer call(Integer integer) { double x = Math.random() * 2 - 1; double y = Math.random() * 2 - 1; return (x * x + y * y < 1) ? 1 : 0; } }).reduce(new Function2() { @Override - public Integer call(Integer integer, Integer integer2) throws Exception { + public Integer call(Integer integer, Integer integer2) { return integer + integer2; } }); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 17f21f6b77..e61b9c4f0e 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -31,11 +31,14 @@ import java.util.Set; /** * Transitive closure on a graph, implemented in Java. */ -public class JavaTC { +public final class JavaTC { - static int numEdges = 200; - static int numVertices = 100; - static Random rand = new Random(42); + private static final int numEdges = 200; + private static final int numVertices = 100; + private static final Random rand = new Random(42); + + private JavaTC() { + } static List> generateGraph() { Set> edges = new HashSet>(numEdges); @@ -43,15 +46,18 @@ public class JavaTC { int from = rand.nextInt(numVertices); int to = rand.nextInt(numVertices); Tuple2 e = new Tuple2(from, to); - if (from != to) edges.add(e); + if (from != to) { + edges.add(e); + } } return new ArrayList>(edges); } static class ProjectFn extends PairFunction>, Integer, Integer> { - static ProjectFn INSTANCE = new ProjectFn(); + static final ProjectFn INSTANCE = new ProjectFn(); + @Override public Tuple2 call(Tuple2> triple) { return new Tuple2(triple._2()._2(), triple._2()._1()); } @@ -76,6 +82,7 @@ public class JavaTC { // Because join() joins on keys, the edges are stored in reversed order. JavaPairRDD edges = tc.map( new PairFunction, Integer, Integer>() { + @Override public Tuple2 call(Tuple2 e) { return new Tuple2(e._2(), e._1()); } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index bd6383e13d..ed4e9b49d0 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -27,8 +27,14 @@ import org.apache.spark.api.java.function.PairFunction; import java.util.Arrays; import java.util.List; +import java.util.regex.Pattern; + +public final class JavaWordCount { + private static final Pattern SPACE = Pattern.compile(" "); + + private JavaWordCount() { + } -public class JavaWordCount { public static void main(String[] args) throws Exception { if (args.length < 2) { System.err.println("Usage: JavaWordCount "); @@ -40,18 +46,21 @@ public class JavaWordCount { JavaRDD lines = ctx.textFile(args[1], 1); JavaRDD words = lines.flatMap(new FlatMapFunction() { + @Override public Iterable call(String s) { - return Arrays.asList(s.split(" ")); + return Arrays.asList(SPACE.split(s)); } }); JavaPairRDD ones = words.map(new PairFunction() { + @Override public Tuple2 call(String s) { return new Tuple2(s, 1); } }); JavaPairRDD counts = ones.reduceByKey(new Function2() { + @Override public Integer call(Integer i1, Integer i2) { return i1 + i2; } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java index 45a0d237da..b33e648147 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java @@ -26,28 +26,35 @@ import org.apache.spark.mllib.recommendation.MatrixFactorizationModel; import org.apache.spark.mllib.recommendation.Rating; import java.util.Arrays; -import java.util.StringTokenizer; +import java.util.regex.Pattern; import scala.Tuple2; /** * Example using MLLib ALS from Java. */ -public class JavaALS { +public final class JavaALS { + + private JavaALS() { + } static class ParseRating extends Function { + private static final Pattern COMMA = Pattern.compile(","); + + @Override public Rating call(String line) { - StringTokenizer tok = new StringTokenizer(line, ","); - int x = Integer.parseInt(tok.nextToken()); - int y = Integer.parseInt(tok.nextToken()); - double rating = Double.parseDouble(tok.nextToken()); + String[] tok = COMMA.split(line); + int x = Integer.parseInt(tok[0]); + int y = Integer.parseInt(tok[1]); + double rating = Double.parseDouble(tok[2]); return new Rating(x, y, rating); } } static class FeaturesToString extends Function, String> { + @Override public String call(Tuple2 element) { - return element._1().toString() + "," + Arrays.toString(element._2()); + return element._1() + "," + Arrays.toString(element._2()); } } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index cd59a139b9..a9db04d525 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -25,20 +25,25 @@ import org.apache.spark.mllib.clustering.KMeans; import org.apache.spark.mllib.clustering.KMeansModel; import java.util.Arrays; -import java.util.StringTokenizer; +import java.util.regex.Pattern; /** * Example using MLLib KMeans from Java. */ -public class JavaKMeans { +public final class JavaKMeans { + + private JavaKMeans() { + } static class ParsePoint extends Function { + private static final Pattern SPACE = Pattern.compile(" "); + + @Override public double[] call(String line) { - StringTokenizer tok = new StringTokenizer(line, " "); - int numTokens = tok.countTokens(); - double[] point = new double[numTokens]; - for (int i = 0; i < numTokens; ++i) { - point[i] = Double.parseDouble(tok.nextToken()); + String[] tok = SPACE.split(line); + double[] point = new double[tok.length]; + for (int i = 0; i < tok.length; ++i) { + point[i] = Double.parseDouble(tok[i]); } return point; } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index 258061c8e6..56341315bf 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -27,22 +27,28 @@ import org.apache.spark.mllib.classification.LogisticRegressionModel; import org.apache.spark.mllib.regression.LabeledPoint; import java.util.Arrays; -import java.util.StringTokenizer; +import java.util.regex.Pattern; /** * Logistic regression based classification using ML Lib. */ -public class JavaLR { +public final class JavaLR { + + private JavaLR() { + } static class ParsePoint extends Function { + private static final Pattern COMMA = Pattern.compile(","); + private static final Pattern SPACE = Pattern.compile(" "); + + @Override public LabeledPoint call(String line) { - String[] parts = line.split(","); + String[] parts = COMMA.split(line); double y = Double.parseDouble(parts[0]); - StringTokenizer tok = new StringTokenizer(parts[1], " "); - int numTokens = tok.countTokens(); - double[] x = new double[numTokens]; - for (int i = 0; i < numTokens; ++i) { - x[i] = Double.parseDouble(tok.nextToken()); + String[] tok = SPACE.split(parts[1]); + double[] x = new double[tok.length]; + for (int i = 0; i < tok.length; ++i) { + x[i] = Double.parseDouble(tok[i]); } return new LabeledPoint(y, x); } diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java index 261813bf2f..bd0bbb56ff 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java @@ -36,7 +36,10 @@ import org.apache.spark.streaming.dstream.SparkFlumeEvent; * creates a server and listens for flume events. * is the port the Flume receiver will listen on. */ -public class JavaFlumeEventCount { +public final class JavaFlumeEventCount { + private JavaFlumeEventCount() { + } + public static void main(String[] args) { if (args.length != 3) { System.err.println("Usage: JavaFlumeEventCount "); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index 22994fb2ec..17eb871908 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -19,6 +19,7 @@ package org.apache.spark.streaming.examples; import java.util.Map; import java.util.HashMap; +import java.util.regex.Pattern; import com.google.common.collect.Lists; import org.apache.spark.api.java.function.FlatMapFunction; @@ -45,7 +46,12 @@ import scala.Tuple2; * zoo03 my-consumer-group topic1,topic2 1` */ -public class JavaKafkaWordCount { +public final class JavaKafkaWordCount { + private static final Pattern SPACE = Pattern.compile(" "); + + private JavaKafkaWordCount() { + } + public static void main(String[] args) { if (args.length < 5) { System.err.println("Usage: KafkaWordCount "); @@ -67,7 +73,7 @@ public class JavaKafkaWordCount { JavaDStream lines = messages.map(new Function, String>() { @Override - public String call(Tuple2 tuple2) throws Exception { + public String call(Tuple2 tuple2) { return tuple2._2(); } }); @@ -75,19 +81,19 @@ public class JavaKafkaWordCount { JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { - return Lists.newArrayList(x.split(" ")); + return Lists.newArrayList(SPACE.split(x)); } }); JavaPairDStream wordCounts = words.map( new PairFunction() { @Override - public Tuple2 call(String s) throws Exception { + public Tuple2 call(String s) { return new Tuple2(s, 1); } }).reduceByKey(new Function2() { @Override - public Integer call(Integer i1, Integer i2) throws Exception { + public Integer call(Integer i1, Integer i2) { return i1 + i2; } }); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index def87c199b..fb090cc262 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -27,6 +27,8 @@ 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.regex.Pattern; + /** * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. * Usage: NetworkWordCount @@ -38,7 +40,12 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; * and then run the example * `$ ./run spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999` */ -public class JavaNetworkWordCount { +public final class JavaNetworkWordCount { + private static final Pattern SPACE = Pattern.compile(" "); + + private JavaNetworkWordCount() { + } + public static void main(String[] args) { if (args.length < 3) { System.err.println("Usage: NetworkWordCount \n" + @@ -56,18 +63,18 @@ public class JavaNetworkWordCount { JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { - return Lists.newArrayList(x.split(" ")); + return Lists.newArrayList(SPACE.split(x)); } }); JavaPairDStream wordCounts = words.map( new PairFunction() { @Override - public Tuple2 call(String s) throws Exception { + public Tuple2 call(String s) { return new Tuple2(s, 1); } }).reduceByKey(new Function2() { @Override - public Integer call(Integer i1, Integer i2) throws Exception { + public Integer call(Integer i1, Integer i2) { return i1 + i2; } }); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java index c8c7389dd1..6be967237c 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java @@ -31,8 +31,11 @@ import java.util.LinkedList; import java.util.List; import java.util.Queue; -public class JavaQueueStream { - public static void main(String[] args) throws InterruptedException { +public final class JavaQueueStream { + private JavaQueueStream() { + } + + public static void main(String[] args) throws Exception { if (args.length < 1) { System.err.println("Usage: JavaQueueStream "); System.exit(1); @@ -62,14 +65,14 @@ public class JavaQueueStream { JavaPairDStream mappedStream = inputStream.map( new PairFunction() { @Override - public Tuple2 call(Integer i) throws Exception { + public Tuple2 call(Integer i) { return new Tuple2(i % 10, 1); } }); JavaPairDStream reducedStream = mappedStream.reduceByKey( new Function2() { @Override - public Integer call(Integer i1, Integer i2) throws Exception { + public Integer call(Integer i1, Integer i2) { return i1 + i2; } }); -- cgit v1.2.3 From 5a3c00c9581f81522a32c0b5d21ba81498c2d9c3 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Mon, 30 Dec 2013 12:23:18 -0800 Subject: Removed redundant TaskSetManager.error() function. This function was leftover from a while ago, and now just passes all calls through to the abort() function, so this commit deletes it. --- .../main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 5 ----- 2 files changed, 1 insertion(+), 6 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index d94b706854..c4ac8337c5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -328,7 +328,7 @@ private[spark] class TaskSchedulerImpl( // Have each task set throw a SparkException with the error for ((taskSetId, manager) <- activeTaskSets) { try { - manager.error(message) + manager.abort(message) } catch { case e: Exception => logError("Exception in error callback", e) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 67ad99a4d7..6dd1469d8f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -548,11 +548,6 @@ private[spark] class TaskSetManager( } } - def error(message: String) { - // Save the error message - abort("Error: " + message) - } - def abort(message: String) { // TODO: Kill running tasks if we were not terminated due to a Mesos error sched.dagScheduler.taskSetFailed(taskSet, message) -- cgit v1.2.3 From a1b438d94de10506dc7dcac54eb331ee2c0479aa Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Thu, 2 Jan 2014 12:34:46 -0800 Subject: Remove erroneous FAILED state for killed tasks. Currently, when tasks are killed, the Executor first sends a status update for the task with a "KILLED" state, and then sends a second status update with a "FAILED" state saying that the task failed due to an exception. The second FAILED state is misleading/unncessary, and occurs due to a NonLocalReturnControl Exception that gets thrown due to the way we kill tasks. This commit eliminates that problem. --- .../scala/org/apache/spark/executor/Executor.scala | 24 ++++++++++++---------- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 3 ++- 2 files changed, 15 insertions(+), 12 deletions(-) (limited to 'core/src') 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 3c92c205ea..e51d274d33 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -141,11 +141,6 @@ private[spark] class Executor( val tr = runningTasks.get(taskId) if (tr != null) { tr.kill() - // We remove the task also in the finally block in TaskRunner.run. - // The reason we need to remove it here is because killTask might be called before the task - // is even launched, and never reaching that finally block. ConcurrentHashMap's remove is - // idempotent. - runningTasks.remove(taskId) } } @@ -167,6 +162,8 @@ private[spark] class Executor( class TaskRunner(execBackend: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) extends Runnable { + object TaskKilledException extends Exception + @volatile private var killed = false @volatile private var task: Task[Any] = _ @@ -200,9 +197,11 @@ private[spark] class Executor( // If this task has been killed before we deserialized it, let's quit now. Otherwise, // continue executing the task. if (killed) { - logInfo("Executor killed task " + taskId) - execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) - return + // Throw an exception rather than returning, because returning within a try{} block + // causes a NonLocalReturnControl exception to be thrown. The NonLocalReturnControl + // exception will be caught by the catch block, leading to an incorrect ExceptionFailure + // for the task. + throw TaskKilledException } attemptedTask = Some(task) @@ -216,9 +215,7 @@ private[spark] class Executor( // If the task has been killed, let's fail it. if (task.killed) { - logInfo("Executor killed task " + taskId) - execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) - return + throw TaskKilledException } val resultSer = SparkEnv.get.serializer.newInstance() @@ -260,6 +257,11 @@ private[spark] class Executor( execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) } + case TaskKilledException => { + logInfo("Executor killed task " + taskId) + execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) + } + case t: Throwable => { val serviceTime = (System.currentTimeMillis() - taskStart).toInt val metrics = attemptedTask.flatMap(t => t.metrics) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index d94b706854..1b67332ff8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -285,7 +285,8 @@ private[spark] class TaskSchedulerImpl( } } case None => - logInfo("Ignoring update from TID " + tid + " because its task set is gone") + logInfo("Ignoring update with state %s from TID %s because its task set is gone" + .format(state, tid)) } } catch { case e: Exception => logError("Exception in statusUpdate", e) -- cgit v1.2.3 From 94f2fffa23436ed66a24c705f88dabe59bf54037 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 3 Jan 2014 14:37:42 +0530 Subject: fixed review comments --- README.md | 19 +++++++++++-------- .../main/scala/org/apache/spark/SparkContext.scala | 7 ++++++- .../test/scala/org/apache/spark/DriverSuite.scala | 9 ++++++--- .../spark/deploy/worker/ExecutorRunnerTest.scala | 8 +++++++- docs/running-on-yarn.md | 4 ++-- project/SparkBuild.scala | 14 +++++++++----- python/lib/py4j-0.8.1.zip | Bin 0 -> 809541 bytes python/lib/py4j0.7.egg | Bin 191756 -> 0 bytes .../scala/org/apache/spark/repl/SparkILoop.scala | 4 +++- 9 files changed, 44 insertions(+), 21 deletions(-) create mode 100644 python/lib/py4j-0.8.1.zip delete mode 100644 python/lib/py4j0.7.egg (limited to 'core/src') diff --git a/README.md b/README.md index 22e7ab8245..8f686743f0 100644 --- a/README.md +++ b/README.md @@ -13,7 +13,7 @@ This README file only contains basic setup instructions. ## Building Spark requires Scala 2.10. The project is built using Simple Build Tool (SBT), -which can be obtained from [here](http://www.scala-sbt.org/release/docs/Getting-Started/Setup.html). To build Spark and its example programs, run: +which can be obtained from [here](http://www.scala-sbt.org). To build Spark and its example programs, run: sbt assembly @@ -38,19 +38,22 @@ locally with one thread, or "local[N]" to run locally with N threads. ## Running tests -### With sbt. (you need sbt installed) -Once you have built spark with `sbt assembly` mentioned in [Building](#Building) section. Test suits can be run as follows on *nix based systems using sbt. +### With sbt (Much faster to run compared to maven) +Once you have built spark with `sbt assembly` mentioned in [Building](#Building) section. Test suits can be run as follows using sbt. -`SPARK_HOME=$(pwd) SPARK_TESTING=1 sbt test` - -TODO: figure out instructions for windows. +`sbt test` ### With maven. +1. Export these necessary environment variables as follows. + + `export SCALA_HOME=` + + `export MAVEN_OPTS="-Xmx1512m -XX:MaxPermSize=512m"` -1. Build assembly by +2. Build assembly by `mvn package -DskipTests` -2. Run tests +3. Run tests `mvn test` ## A Note About Hadoop Versions diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 4d6a97e255..c6f6883b01 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -169,10 +169,15 @@ 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_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING"); + for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS"); value <- Option(System.getenv(key))) { executorEnvs(key) = value } + // A workaround for SPARK_TESTING and SPARK_HOME + for { (envKey, propKey) <- Seq(("SPARK_HOME", "spark.home"), ("SPARK_TESTING", "spark.testing")) + value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { + executorEnvs(envKey) = value + } // Since memory can be set with a system property too, use that executorEnvs("SPARK_MEM") = executorMemory + "m" executorEnvs ++= conf.getExecutorEnv diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 6d1695eae7..89c5631ad8 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -30,13 +30,16 @@ import org.apache.spark.util.Utils class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { - assert(System.getenv("SPARK_HOME") != null) + val sparkHome = Option(System.getenv("SPARK_HOME")) + .orElse(Option(System.getProperty("spark.home"))).get // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing" val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => failAfter(60 seconds) { - Utils.execute(Seq("./spark-class", "org.apache.spark.DriverWithoutCleanup", master), - new File(System.getenv("SPARK_HOME"))) + Utils.executeAndGetOutput( + Seq("./spark-class", "org.apache.spark.DriverWithoutCleanup", master), + new File(sparkHome), + Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 4cb4ddc9cd..894a72284b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -18,13 +18,19 @@ package org.apache.spark.deploy.worker import java.io.File + +import scala.util.Try + import org.scalatest.FunSuite + import org.apache.spark.deploy.{ExecutorState, Command, ApplicationDescription} + class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) - val sparkHome = sys.env("SPARK_HOME") + val sparkHome = Try(sys.env("SPARK_HOME")).toOption + .orElse(Option(System.getProperty("spark.home"))).get val appDesc = new ApplicationDescription("app name", 8, 500, Command("foo", Seq(),Map()), sparkHome, "appUiUrl") val appId = "12345-worker321-9876" diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 13d5fd3685..aded6430b3 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -12,7 +12,7 @@ was added to Spark in version 0.6.0, and improved in 0.7.0 and 0.8.0. 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: - SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt assembly + SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt assembly The assembled JAR will be something like this: `./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly_{{site.SPARK_VERSION}}-hadoop2.0.5.jar`. @@ -72,7 +72,7 @@ The command to launch the YARN Client is as follows: For example: # Build the Spark assembly JAR and the Spark examples JAR - $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt assembly + $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt assembly # Configure logging $ cp conf/log4j.properties.template conf/log4j.properties diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b335b5a20a..8290e7cf43 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -20,6 +20,7 @@ import sbt.Classpaths.publishTask import Keys._ import sbtassembly.Plugin._ import AssemblyKeys._ +import scala.util.Properties // For Sonatype publishing //import com.jsuereth.pgp.sbtplugin.PgpKeys._ @@ -68,10 +69,12 @@ object SparkBuild extends Build { // 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") - + val sparkHome = System.getProperty("user.dir") + System.setProperty("spark.home", sparkHome) + System.setProperty("spark.testing", "1") // Allows build configuration to be set through environment variables - lazy val hadoopVersion = scala.util.Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) - lazy val isNewHadoop = scala.util.Properties.envOrNone("SPARK_IS_NEW_HADOOP") match { + lazy val hadoopVersion = Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) + lazy val isNewHadoop = Properties.envOrNone("SPARK_IS_NEW_HADOOP") match { case None => { val isNewHadoopVersion = "2.[2-9]+".r.findFirstIn(hadoopVersion).isDefined (isNewHadoopVersion|| DEFAULT_IS_NEW_HADOOP) @@ -79,7 +82,7 @@ object SparkBuild extends Build { case Some(v) => v.toBoolean } - lazy val isYarnEnabled = scala.util.Properties.envOrNone("SPARK_YARN") match { + lazy val isYarnEnabled = Properties.envOrNone("SPARK_YARN") match { case None => DEFAULT_YARN case Some(v) => v.toBoolean } @@ -112,8 +115,9 @@ object SparkBuild extends Build { // Fork new JVMs for tests and set Java options for those fork := true, + javaOptions += "-Dspark.home=" + sparkHome, + javaOptions += "-Dspark.testing=1", javaOptions += "-Xmx3g", - // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), diff --git a/python/lib/py4j-0.8.1.zip b/python/lib/py4j-0.8.1.zip new file mode 100644 index 0000000000..3231e31164 Binary files /dev/null and b/python/lib/py4j-0.8.1.zip differ diff --git a/python/lib/py4j0.7.egg b/python/lib/py4j0.7.egg deleted file mode 100644 index f8a339d8ee..0000000000 Binary files a/python/lib/py4j0.7.egg and /dev/null differ diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index f108c70f21..7e54421b52 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -942,12 +942,14 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, val conf = new SparkConf() .setMaster(master) .setAppName("Spark shell") - .setSparkHome(System.getenv("SPARK_HOME")) .setJars(jars) .set("spark.repl.class.uri", intp.classServer.uri) if (execUri != null) { conf.set("spark.executor.uri", execUri) } + if (System.getenv("SPARK_HOME") != null) { + conf.setSparkHome(System.getenv("SPARK_HOME")) + } sparkContext = new SparkContext(conf) echo("Created spark context..") sparkContext -- cgit v1.2.3 From bc311bb826b5548b9c4c55320711f3b18dc19397 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 3 Jan 2014 14:52:37 +0530 Subject: Restored the previously removed test --- core/src/test/scala/org/apache/spark/FileServerSuite.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index a15c3751c2..506f7484fb 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -55,7 +55,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { in.close() jar.close() stream.close() - testJarFile = tmpFile2.getAbsolutePath + testJarFile = tmpFile2.toURI.toURL.toString } override def beforeEach() { @@ -142,4 +142,15 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } } + test ("Dynamically adding JARS on a standalone cluster using local: URL") { + sc = new SparkContext("local-cluster[1,1,512]", "test") + sc.addJar(testJarFile.replace("file", "local")) + val testData = Array((1,1)) + sc.parallelize(testData).foreach { (x) => + if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite2.txt") == null) { + throw new SparkException("jar not added") + } + } + } + } -- cgit v1.2.3 From 74ba97fcf75ae61833295a896dcdf775036ae1d8 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 3 Jan 2014 15:08:01 +0530 Subject: sbin/spark-class* -> bin/spark-class* --- bin/run-example | 2 +- bin/run-example2.cmd | 2 +- bin/spark-class | 154 +++++++++++++++++++++ bin/spark-class.cmd | 23 +++ bin/spark-class2.cmd | 85 ++++++++++++ bin/spark-shell | 4 +- .../mesos/CoarseMesosSchedulerBackend.scala | 4 +- .../org/apache/spark/ui/UIWorkloadGenerator.scala | 4 +- .../test/scala/org/apache/spark/DriverSuite.scala | 2 +- docs/running-on-yarn.md | 4 +- docs/spark-standalone.md | 2 +- python/pyspark/java_gateway.py | 2 +- sbin/spark-class | 154 --------------------- sbin/spark-class.cmd | 23 --- sbin/spark-class2.cmd | 85 ------------ sbin/spark-daemon.sh | 2 +- sbin/spark-executor | 2 +- 17 files changed, 277 insertions(+), 277 deletions(-) create mode 100755 bin/spark-class create mode 100644 bin/spark-class.cmd create mode 100644 bin/spark-class2.cmd delete mode 100755 sbin/spark-class delete mode 100644 sbin/spark-class.cmd delete mode 100644 sbin/spark-class2.cmd (limited to 'core/src') diff --git a/bin/run-example b/bin/run-example index a3976beb12..f2699c38a9 100755 --- a/bin/run-example +++ b/bin/run-example @@ -36,7 +36,7 @@ if [ -e "$FWDIR/conf/spark-env.sh" ] ; then fi if [ -z "$1" ]; then - echo "Usage: bin/run-example []" >&2 + echo "Usage: run-example []" >&2 exit 1 fi diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd index d9c10545e7..6861334cb0 100644 --- a/bin/run-example2.cmd +++ b/bin/run-example2.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: bin/run-example ^ [^] + echo Usage: run-example ^ [^] goto exit :arg_given diff --git a/bin/spark-class b/bin/spark-class new file mode 100755 index 0000000000..4e440d8729 --- /dev/null +++ b/bin/spark-class @@ -0,0 +1,154 @@ +#!/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. +# + +cygwin=false +case "`uname`" in + CYGWIN*) cygwin=true;; +esac + +SCALA_VERSION=2.10 + +# 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: spark-class []" >&2 + exit 1 +fi + +# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable +# values for that; it doesn't need a lot +if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then + SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} + SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" + # 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 + + +# Add java opts for master, worker, executor. The opts maybe null +case "$1" in + 'org.apache.spark.deploy.master.Master') + OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS" + ;; + 'org.apache.spark.deploy.worker.Worker') + OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS" + ;; + 'org.apache.spark.executor.CoarseGrainedExecutorBackend') + OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS" + ;; + 'org.apache.spark.executor.MesosExecutorBackend') + OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS" + ;; + 'org.apache.spark.repl.Main') + OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS" + ;; +esac + +# Find the 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 + +# 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 +JAVA_OPTS="$OUR_JAVA_OPTS" +JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" +JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM" +# Load extra JAVA_OPTS from conf/java-opts, if it exists +if [ -e "$FWDIR/conf/java-opts" ] ; then + JAVA_OPTS="$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! + +if [ ! -f "$FWDIR/RELEASE" ]; then + # Exit if the user hasn't compiled Spark + num_jars=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar" | wc -l) + jars_list=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar") + if [ "$num_jars" -eq "0" ]; then + echo "Failed to find Spark assembly in $FWDIR/assembly/target/scala-$SCALA_VERSION/" >&2 + echo "You need to build Spark with 'sbt/sbt assembly' before running this program." >&2 + exit 1 + fi + if [ "$num_jars" -gt "1" ]; then + echo "Found multiple Spark assembly jars in $FWDIR/assembly/target/scala-$SCALA_VERSION:" >&2 + echo "$jars_list" + echo "Please remove all but one jar." + exit 1 + fi +fi + +TOOLS_DIR="$FWDIR"/tools +SPARK_TOOLS_JAR="" +if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then + # Use the JAR from the SBT build + export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar` +fi +if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then + # Use the JAR from the Maven build + # TODO: this also needs to become an assembly! + export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar` +fi + +# Compute classpath using external script +CLASSPATH=`$FWDIR/sbin/compute-classpath.sh` + +if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then + CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" +fi + +if $cygwin; then + CLASSPATH=`cygpath -wp $CLASSPATH` + if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then + export SPARK_TOOLS_JAR=`cygpath -w $SPARK_TOOLS_JAR` + fi +fi +export CLASSPATH + +if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then + echo -n "Spark Command: " + echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" + echo "========================================" + echo +fi + +exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" + + diff --git a/bin/spark-class.cmd b/bin/spark-class.cmd new file mode 100644 index 0000000000..19850db9e1 --- /dev/null +++ b/bin/spark-class.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 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/bin/spark-class2.cmd b/bin/spark-class2.cmd new file mode 100644 index 0000000000..460e661476 --- /dev/null +++ b/bin/spark-class2.cmd @@ -0,0 +1,85 @@ +@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.10 + +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: spark-class ^ [^] + goto exit +:arg_given + +set RUNNING_DAEMON=0 +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 +set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true +if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY% +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 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 Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! + +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 + +set TOOLS_DIR=%FWDIR%tools +set SPARK_TOOLS_JAR= +for %%d in ("%TOOLS_DIR%\target\scala-%SCALA_VERSION%\spark-tools*assembly*.jar") do ( + set SPARK_TOOLS_JAR=%%d +) + +rem Compute classpath using external script +set DONT_PRINT_CLASSPATH=1 +call "%FWDIR%sbin\compute-classpath.cmd" +set DONT_PRINT_CLASSPATH=0 +set CLASSPATH=%CLASSPATH%;%SPARK_TOOLS_JAR% + +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/bin/spark-shell b/bin/spark-shell index bc7386db4d..e6885b51ef 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -90,10 +90,10 @@ if $cygwin; then # "Backspace sends ^H" setting in "Keys" section of the Mintty options # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 - $FWDIR/sbin/spark-class -Djline.terminal=unix $OPTIONS org.apache.spark.repl.Main "$@" + $FWDIR/bin/spark-class -Djline.terminal=unix $OPTIONS org.apache.spark.repl.Main "$@" stty icanon echo > /dev/null 2>&1 else - $FWDIR/sbin/spark-class $OPTIONS org.apache.spark.repl.Main "$@" + $FWDIR/bin/spark-class $OPTIONS org.apache.spark.repl.Main "$@" fi # record the exit status lest it be overwritten: diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 0494ca8726..d46fceba89 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -127,7 +127,7 @@ private[spark] class CoarseMesosSchedulerBackend( CoarseGrainedSchedulerBackend.ACTOR_NAME) val uri = conf.get("spark.executor.uri", null) if (uri == null) { - val runScript = new File(sparkHome, "./sbin/spark-class").getCanonicalPath + val runScript = new File(sparkHome, "./bin/spark-class").getCanonicalPath command.setValue( "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format( runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) @@ -136,7 +136,7 @@ private[spark] class CoarseMesosSchedulerBackend( // glob the directory "correctly". val basename = uri.split('/').last.split('.').head command.setValue( - "cd %s*; ./sbin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d" + "cd %s*; ./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %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/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index f207627e52..6ba15187d9 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -27,7 +27,7 @@ import org.apache.spark.scheduler.SchedulingMode /** * Continuously generates jobs that expose various features of the WebUI (internal testing tool). * - * Usage: ./run spark.ui.UIWorkloadGenerator [master] + * Usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR] */ private[spark] object UIWorkloadGenerator { @@ -36,7 +36,7 @@ private[spark] object UIWorkloadGenerator { def main(args: Array[String]) { if (args.length < 2) { - println("usage: ./sbin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") + println("usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") System.exit(1) } diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index c37fd9ab62..605588f7f6 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -35,7 +35,7 @@ class DriverSuite extends FunSuite with Timeouts { val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => failAfter(60 seconds) { - Utils.execute(Seq("./sbin/spark-class", "org.apache.spark.DriverWithoutCleanup", master), + Utils.execute(Seq("./bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master), new File(System.getenv("SPARK_HOME"))) } } diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index e2b21f9cde..e21812378c 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -54,7 +54,7 @@ There are two scheduler mode that can be used to launch spark application on YAR The command to launch the YARN Client is as follows: - SPARK_JAR= ./sbin/spark-class org.apache.spark.deploy.yarn.Client \ + SPARK_JAR= ./bin/spark-class org.apache.spark.deploy.yarn.Client \ --jar \ --class \ --args \ @@ -79,7 +79,7 @@ For example: # Submit Spark's ApplicationMaster to YARN's ResourceManager, and instruct Spark to run the SparkPi example $ SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - ./sbin/spark-class org.apache.spark.deploy.yarn.Client \ + ./bin/spark-class org.apache.spark.deploy.yarn.Client \ --jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ --class org.apache.spark.examples.SparkPi \ --args yarn-standalone \ diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index baa0a062f7..c851833a18 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -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: - ./sbin/spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT + ./bin/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/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 7243ee6861..c15add5237 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -31,7 +31,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 on_windows = platform.system() == "Windows" - script = "./sbin/spark-class.cmd" if on_windows else "./sbin/spark-class" + script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class" command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer", "--die-on-broken-pipe", "0"] if not on_windows: diff --git a/sbin/spark-class b/sbin/spark-class deleted file mode 100755 index 4e440d8729..0000000000 --- a/sbin/spark-class +++ /dev/null @@ -1,154 +0,0 @@ -#!/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. -# - -cygwin=false -case "`uname`" in - CYGWIN*) cygwin=true;; -esac - -SCALA_VERSION=2.10 - -# 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: spark-class []" >&2 - exit 1 -fi - -# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable -# values for that; it doesn't need a lot -if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then - SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} - SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" - # 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 - - -# Add java opts for master, worker, executor. The opts maybe null -case "$1" in - 'org.apache.spark.deploy.master.Master') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS" - ;; - 'org.apache.spark.deploy.worker.Worker') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS" - ;; - 'org.apache.spark.executor.CoarseGrainedExecutorBackend') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS" - ;; - 'org.apache.spark.executor.MesosExecutorBackend') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS" - ;; - 'org.apache.spark.repl.Main') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS" - ;; -esac - -# Find the 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 - -# 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 -JAVA_OPTS="$OUR_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" -JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM" -# Load extra JAVA_OPTS from conf/java-opts, if it exists -if [ -e "$FWDIR/conf/java-opts" ] ; then - JAVA_OPTS="$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! - -if [ ! -f "$FWDIR/RELEASE" ]; then - # Exit if the user hasn't compiled Spark - num_jars=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar" | wc -l) - jars_list=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar") - if [ "$num_jars" -eq "0" ]; then - echo "Failed to find Spark assembly in $FWDIR/assembly/target/scala-$SCALA_VERSION/" >&2 - echo "You need to build Spark with 'sbt/sbt assembly' before running this program." >&2 - exit 1 - fi - if [ "$num_jars" -gt "1" ]; then - echo "Found multiple Spark assembly jars in $FWDIR/assembly/target/scala-$SCALA_VERSION:" >&2 - echo "$jars_list" - echo "Please remove all but one jar." - exit 1 - fi -fi - -TOOLS_DIR="$FWDIR"/tools -SPARK_TOOLS_JAR="" -if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then - # Use the JAR from the SBT build - export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar` -fi -if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then - # Use the JAR from the Maven build - # TODO: this also needs to become an assembly! - export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar` -fi - -# Compute classpath using external script -CLASSPATH=`$FWDIR/sbin/compute-classpath.sh` - -if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then - CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" -fi - -if $cygwin; then - CLASSPATH=`cygpath -wp $CLASSPATH` - if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then - export SPARK_TOOLS_JAR=`cygpath -w $SPARK_TOOLS_JAR` - fi -fi -export CLASSPATH - -if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then - echo -n "Spark Command: " - echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" - echo "========================================" - echo -fi - -exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" - - diff --git a/sbin/spark-class.cmd b/sbin/spark-class.cmd deleted file mode 100644 index 19850db9e1..0000000000 --- a/sbin/spark-class.cmd +++ /dev/null @@ -1,23 +0,0 @@ -@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 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/sbin/spark-class2.cmd b/sbin/spark-class2.cmd deleted file mode 100644 index 460e661476..0000000000 --- a/sbin/spark-class2.cmd +++ /dev/null @@ -1,85 +0,0 @@ -@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.10 - -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: spark-class ^ [^] - goto exit -:arg_given - -set RUNNING_DAEMON=0 -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 -set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true -if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY% -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 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 Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! - -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 - -set TOOLS_DIR=%FWDIR%tools -set SPARK_TOOLS_JAR= -for %%d in ("%TOOLS_DIR%\target\scala-%SCALA_VERSION%\spark-tools*assembly*.jar") do ( - set SPARK_TOOLS_JAR=%%d -) - -rem Compute classpath using external script -set DONT_PRINT_CLASSPATH=1 -call "%FWDIR%sbin\compute-classpath.cmd" -set DONT_PRINT_CLASSPATH=0 -set CLASSPATH=%CLASSPATH%;%SPARK_TOOLS_JAR% - -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/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index ca6b893b9b..2be2b3d7c0 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -147,7 +147,7 @@ case $startStop in spark_rotate_log "$log" echo starting $command, logging to $log cd "$SPARK_PREFIX" - nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/sbin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & + nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & newpid=$! echo $newpid > $pid sleep 2 diff --git a/sbin/spark-executor b/sbin/spark-executor index 214e00f6f8..de5bfab563 100755 --- a/sbin/spark-executor +++ b/sbin/spark-executor @@ -20,4 +20,4 @@ FWDIR="$(cd `dirname $0`/..; pwd)" echo "Running spark-executor with framework dir = $FWDIR" -exec $FWDIR/sbin/spark-class org.apache.spark.executor.MesosExecutorBackend +exec $FWDIR/bin/spark-class org.apache.spark.executor.MesosExecutorBackend -- cgit v1.2.3 From 9ae382c363202eac8ef0b8e0fe1a7eab3de545b1 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 3 Jan 2014 15:12:09 +0530 Subject: sbin/compute-classpath* bin/compute-classpath* --- bin/compute-classpath.cmd | 69 ++++++++++++++++++++ bin/compute-classpath.sh | 75 ++++++++++++++++++++++ bin/run-example | 2 +- bin/spark-class | 2 +- .../spark/deploy/worker/ExecutorRunner.scala | 2 +- sbin/compute-classpath.cmd | 69 -------------------- sbin/compute-classpath.sh | 75 ---------------------- 7 files changed, 147 insertions(+), 147 deletions(-) create mode 100644 bin/compute-classpath.cmd create mode 100755 bin/compute-classpath.sh delete mode 100644 sbin/compute-classpath.cmd delete mode 100755 sbin/compute-classpath.sh (limited to 'core/src') diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd new file mode 100644 index 0000000000..4f60bff19c --- /dev/null +++ b/bin/compute-classpath.cmd @@ -0,0 +1,69 @@ +@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. + +set SCALA_VERSION=2.10 + +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" + +rem Build up classpath +set 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 +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 A bit of a hack to allow calling this script within run2.cmd without seeing output +if "%DONT_PRINT_CLASSPATH%"=="1" goto exit + +echo %CLASSPATH% + +:exit diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh new file mode 100755 index 0000000000..0c82310421 --- /dev/null +++ b/bin/compute-classpath.sh @@ -0,0 +1,75 @@ +#!/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. +# + +# 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.10 + +# 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 + +# Build up classpath +CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" + +# First check if we have a dependencies jar. If so, include binary classes with the deps jar +if [ -f "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar ]; then + CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/classes" + + DEPS_ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar` + CLASSPATH="$CLASSPATH:$DEPS_ASSEMBLY_JAR" +else + # Else use spark-assembly jar from either RELEASE or assembly directory + 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*hadoop*.jar` + fi + CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" +fi + +# 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 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 + CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR" +fi +if [ "x" != "x$YARN_CONF_DIR" ]; then + CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" +fi + +echo "$CLASSPATH" diff --git a/bin/run-example b/bin/run-example index f2699c38a9..6c5d4a6a8f 100755 --- a/bin/run-example +++ b/bin/run-example @@ -61,7 +61,7 @@ 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/sbin/compute-classpath.sh` +CLASSPATH=`$FWDIR/bin/compute-classpath.sh` CLASSPATH="$SPARK_EXAMPLES_JAR:$CLASSPATH" if $cygwin; then diff --git a/bin/spark-class b/bin/spark-class index 4e440d8729..c4225a392d 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -128,7 +128,7 @@ if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then fi # Compute classpath using external script -CLASSPATH=`$FWDIR/sbin/compute-classpath.sh` +CLASSPATH=`$FWDIR/bin/compute-classpath.sh` if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" 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 d4084820f6..fff9cb60c7 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 @@ -122,7 +122,7 @@ 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 + "/sbin/compute-classpath" + ext), + Seq(sparkHome + "/bin/compute-classpath" + ext), extraEnvironment=appDesc.command.environment) Seq("-cp", classPath) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts diff --git a/sbin/compute-classpath.cmd b/sbin/compute-classpath.cmd deleted file mode 100644 index 4f60bff19c..0000000000 --- a/sbin/compute-classpath.cmd +++ /dev/null @@ -1,69 +0,0 @@ -@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. - -set SCALA_VERSION=2.10 - -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" - -rem Build up classpath -set 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 -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 A bit of a hack to allow calling this script within run2.cmd without seeing output -if "%DONT_PRINT_CLASSPATH%"=="1" goto exit - -echo %CLASSPATH% - -:exit diff --git a/sbin/compute-classpath.sh b/sbin/compute-classpath.sh deleted file mode 100755 index 0c82310421..0000000000 --- a/sbin/compute-classpath.sh +++ /dev/null @@ -1,75 +0,0 @@ -#!/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. -# - -# 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.10 - -# 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 - -# Build up classpath -CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" - -# First check if we have a dependencies jar. If so, include binary classes with the deps jar -if [ -f "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar ]; then - CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/classes" - - DEPS_ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar` - CLASSPATH="$CLASSPATH:$DEPS_ASSEMBLY_JAR" -else - # Else use spark-assembly jar from either RELEASE or assembly directory - 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*hadoop*.jar` - fi - CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" -fi - -# 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 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 - CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR" -fi -if [ "x" != "x$YARN_CONF_DIR" ]; then - CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" -fi - -echo "$CLASSPATH" -- cgit v1.2.3 From 9e6f3bdcda1ab48159afa4f54b64d05e42a8688e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 3 Jan 2014 17:32:25 -0800 Subject: Changes on top of Prashant's patch. Closes #316 --- README.md | 19 +---- .../main/scala/org/apache/spark/SparkContext.scala | 3 +- .../test/scala/org/apache/spark/DriverSuite.scala | 3 +- .../scala/org/apache/spark/FileServerSuite.scala | 77 +++++++++------------ .../spark/deploy/worker/ExecutorRunnerTest.scala | 6 +- make-distribution.sh | 3 +- project/SparkBuild.scala | 1 + python/lib/py4j-0.8.1-src.zip | Bin 0 -> 37662 bytes python/lib/py4j-0.8.1.zip | Bin 809541 -> 0 bytes python/pyspark/__init__.py | 2 +- 10 files changed, 42 insertions(+), 72 deletions(-) create mode 100644 python/lib/py4j-0.8.1-src.zip delete mode 100644 python/lib/py4j-0.8.1.zip (limited to 'core/src') diff --git a/README.md b/README.md index 8f686743f0..873ec9882f 100644 --- a/README.md +++ b/README.md @@ -13,7 +13,7 @@ This README file only contains basic setup instructions. ## Building Spark requires Scala 2.10. The project is built using Simple Build Tool (SBT), -which can be obtained from [here](http://www.scala-sbt.org). To build Spark and its example programs, run: +which can be obtained [here](http://www.scala-sbt.org). To build Spark and its example programs, run: sbt assembly @@ -38,24 +38,11 @@ locally with one thread, or "local[N]" to run locally with N threads. ## Running tests -### With sbt (Much faster to run compared to maven) -Once you have built spark with `sbt assembly` mentioned in [Building](#Building) section. Test suits can be run as follows using sbt. +Testing first requires [Building](#Building) Spark. Once Spark is built, tests +can be run using: `sbt test` -### With maven. -1. Export these necessary environment variables as follows. - - `export SCALA_HOME=` - - `export MAVEN_OPTS="-Xmx1512m -XX:MaxPermSize=512m"` - -2. Build assembly by -`mvn package -DskipTests` - -3. Run tests -`mvn test` - ## A Note About Hadoop Versions Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c6f6883b01..e80e43af6d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -173,7 +173,8 @@ class SparkContext( value <- Option(System.getenv(key))) { executorEnvs(key) = value } - // A workaround for SPARK_TESTING and SPARK_HOME + // Convert java options to env vars as a work around + // since we can't set env vars directly in sbt. for { (envKey, propKey) <- Seq(("SPARK_HOME", "spark.home"), ("SPARK_TESTING", "spark.testing")) value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { executorEnvs(envKey) = value diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 89c5631ad8..7e1e55fa3b 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -30,8 +30,7 @@ import org.apache.spark.util.Utils class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { - val sparkHome = Option(System.getenv("SPARK_HOME")) - .orElse(Option(System.getProperty("spark.home"))).get + val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing" val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 506f7484fb..a2eb9a4e84 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -27,54 +27,39 @@ import org.scalatest.FunSuite class FileServerSuite extends FunSuite with LocalSparkContext { @transient var tmpFile: File = _ - @transient var testJarFile: String = _ - + @transient var tmpJarUrl: String = _ override def beforeAll() { super.beforeAll() - val buffer = new Array[Byte](10240) - val tmpdir = new File(Files.createTempDir(), "test") - tmpdir.mkdir() - val tmpJarEntry = new File(tmpdir, "FileServerSuite2.txt") - val pw = new PrintWriter(tmpJarEntry) - pw.println("test String in the file named FileServerSuite2.txt") + val tmpDir = new File(Files.createTempDir(), "test") + tmpDir.mkdir() + + val textFile = new File(tmpDir, "FileServerSuite.txt") + val pw = new PrintWriter(textFile) + pw.println("100") pw.close() - // The ugliest code possible, was translated from java. - val tmpFile2 = new File(tmpdir, "test.jar") - val stream = new FileOutputStream(tmpFile2) - val jar = new JarOutputStream(stream, new java.util.jar.Manifest()) - val jarAdd = new JarEntry(tmpJarEntry.getName) - jarAdd.setTime(tmpJarEntry.lastModified) - jar.putNextEntry(jarAdd) - val in = new FileInputStream(tmpJarEntry) + + val jarFile = new File(tmpDir, "test.jar") + val jarStream = new FileOutputStream(jarFile) + val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) + + val jarEntry = new JarEntry(textFile.getName) + jar.putNextEntry(jarEntry) + + val in = new FileInputStream(textFile) + val buffer = new Array[Byte](10240) var nRead = 0 - while (nRead <= 0) { + while (nRead <= 0) { nRead = in.read(buffer, 0, buffer.length) jar.write(buffer, 0, nRead) } + in.close() jar.close() - stream.close() - testJarFile = tmpFile2.toURI.toURL.toString - } - - override def beforeEach() { - super.beforeEach() - // Create a sample text file - val tmpdir = new File(Files.createTempDir(), "test") - tmpdir.mkdir() - tmpFile = new File(tmpdir, "FileServerSuite.txt") - val pw = new PrintWriter(tmpFile) - pw.println("100") - pw.close() - } + jarStream.close() - override def afterEach() { - super.afterEach() - // Clean up downloaded file - if (tmpFile.exists) { - tmpFile.delete() - } + tmpFile = textFile + tmpJarUrl = jarFile.toURI.toURL.toString } test("Distributing files locally") { @@ -108,10 +93,10 @@ class FileServerSuite extends FunSuite with LocalSparkContext { test ("Dynamically adding JARS locally") { sc = new SparkContext("local[4]", "test") - sc.addJar(testJarFile) + sc.addJar(tmpJarUrl) val testData = Array((1, 1)) - sc.parallelize(testData).foreach { (x) => - if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite2.txt") == null) { + sc.parallelize(testData).foreach { x => + if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) { throw new SparkException("jar not added") } } @@ -133,10 +118,10 @@ class FileServerSuite extends FunSuite with LocalSparkContext { test ("Dynamically adding JARS on a standalone cluster") { sc = new SparkContext("local-cluster[1,1,512]", "test") - sc.addJar(testJarFile) + sc.addJar(tmpJarUrl) val testData = Array((1,1)) - sc.parallelize(testData).foreach { (x) => - if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite2.txt") == null) { + sc.parallelize(testData).foreach { x => + if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) { throw new SparkException("jar not added") } } @@ -144,10 +129,10 @@ class FileServerSuite extends FunSuite with LocalSparkContext { test ("Dynamically adding JARS on a standalone cluster using local: URL") { sc = new SparkContext("local-cluster[1,1,512]", "test") - sc.addJar(testJarFile.replace("file", "local")) + sc.addJar(tmpJarUrl.replace("file", "local")) val testData = Array((1,1)) - sc.parallelize(testData).foreach { (x) => - if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite2.txt") == null) { + sc.parallelize(testData).foreach { x => + if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) { throw new SparkException("jar not added") } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 894a72284b..f58b1ee05a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -19,18 +19,14 @@ package org.apache.spark.deploy.worker import java.io.File -import scala.util.Try - import org.scalatest.FunSuite import org.apache.spark.deploy.{ExecutorState, Command, ApplicationDescription} - class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) - val sparkHome = Try(sys.env("SPARK_HOME")).toOption - .orElse(Option(System.getProperty("spark.home"))).get + val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.env.get("spark.home")).get val appDesc = new ApplicationDescription("app name", 8, 500, Command("foo", Seq(),Map()), sparkHome, "appUiUrl") val appId = "12345-worker321-9876" diff --git a/make-distribution.sh b/make-distribution.sh index a2c8e64597..8ae8a4cf74 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -45,7 +45,8 @@ DISTDIR="$FWDIR/dist" export TERM=dumb # Prevents color codes in SBT output if ! test `which sbt` ;then - echo -e "You need sbt installed and available on path, please follow the instructions here: http://www.scala-sbt.org/release/docs/Getting-Started/Setup.html" + echo -e "You need sbt installed and available on your path." + echo -e "Download sbt from http://www.scala-sbt.org/" exit -1; fi diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8290e7cf43..5f57c964bd 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -72,6 +72,7 @@ object SparkBuild extends Build { val sparkHome = System.getProperty("user.dir") System.setProperty("spark.home", sparkHome) System.setProperty("spark.testing", "1") + // Allows build configuration to be set through environment variables lazy val hadoopVersion = Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) lazy val isNewHadoop = Properties.envOrNone("SPARK_IS_NEW_HADOOP") match { diff --git a/python/lib/py4j-0.8.1-src.zip b/python/lib/py4j-0.8.1-src.zip new file mode 100644 index 0000000000..2069a328d1 Binary files /dev/null and b/python/lib/py4j-0.8.1-src.zip differ diff --git a/python/lib/py4j-0.8.1.zip b/python/lib/py4j-0.8.1.zip deleted file mode 100644 index 3231e31164..0000000000 Binary files a/python/lib/py4j-0.8.1.zip and /dev/null differ diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 2b2c3a061a..a51d5af79b 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -40,7 +40,7 @@ Public classes: import sys import os -sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j0.7.egg")) +sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j-0.8.1-src.zip")) from pyspark.conf import SparkConf -- cgit v1.2.3 From ad35c1a5f2bbc44c077ccf1adb41910dc7ef0029 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Sat, 4 Jan 2014 11:42:17 -0600 Subject: Fix handling of empty SPARK_EXAMPLES_JAR --- core/src/main/scala/org/apache/spark/SparkConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 98343e9532..7073a999a9 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -67,7 +67,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { /** Set JAR files to distribute to the cluster. */ def setJars(jars: Seq[String]): SparkConf = { - set("spark.jars", jars.mkString(",")) + set("spark.jars", jars.filter(_ != null).mkString(",")) } /** Set JAR files to distribute to the cluster. (Java-friendly version.) */ -- cgit v1.2.3 From 79f52809c836d08023aa5ca99a467d3a311a7359 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 5 Jan 2014 11:43:00 -0800 Subject: Removing SPARK_EXAMPLES_JAR in the code --- .../apache/spark/api/java/JavaSparkContext.scala | 6 ++++++ .../java/org/apache/spark/examples/JavaHdfsLR.java | 2 +- .../java/org/apache/spark/examples/JavaKMeans.java | 2 +- .../org/apache/spark/examples/JavaLogQuery.java | 2 +- .../org/apache/spark/examples/JavaPageRank.java | 3 ++- .../org/apache/spark/examples/JavaSparkPi.java | 2 +- .../java/org/apache/spark/examples/JavaTC.java | 2 +- .../org/apache/spark/examples/JavaWordCount.java | 2 +- .../org/apache/spark/mllib/examples/JavaALS.java | 2 +- .../apache/spark/mllib/examples/JavaKMeans.java | 2 +- .../org/apache/spark/mllib/examples/JavaLR.java | 2 +- .../streaming/examples/JavaFlumeEventCount.java | 3 ++- .../streaming/examples/JavaKafkaWordCount.java | 3 ++- .../streaming/examples/JavaNetworkWordCount.java | 3 ++- .../spark/streaming/examples/JavaQueueStream.java | 2 +- .../org/apache/spark/examples/BroadcastTest.scala | 2 +- .../spark/examples/ExceptionHandlingTest.scala | 2 +- .../org/apache/spark/examples/GroupByTest.scala | 2 +- .../org/apache/spark/examples/HBaseTest.scala | 2 +- .../scala/org/apache/spark/examples/HdfsTest.scala | 2 +- .../scala/org/apache/spark/examples/LogQuery.scala | 2 +- .../apache/spark/examples/MultiBroadcastTest.scala | 2 +- .../spark/examples/SimpleSkewedGroupByTest.scala | 2 +- .../apache/spark/examples/SkewedGroupByTest.scala | 2 +- .../scala/org/apache/spark/examples/SparkALS.scala | 2 +- .../org/apache/spark/examples/SparkHdfsLR.scala | 2 +- .../org/apache/spark/examples/SparkKMeans.scala | 2 +- .../scala/org/apache/spark/examples/SparkLR.scala | 2 +- .../org/apache/spark/examples/SparkPageRank.scala | 2 +- .../scala/org/apache/spark/examples/SparkPi.scala | 2 +- .../scala/org/apache/spark/examples/SparkTC.scala | 2 +- .../spark/streaming/examples/ActorWordCount.scala | 2 +- .../spark/streaming/examples/FlumeEventCount.scala | 2 +- .../spark/streaming/examples/HdfsWordCount.scala | 2 +- .../spark/streaming/examples/KafkaWordCount.scala | 2 +- .../spark/streaming/examples/MQTTWordCount.scala | 2 +- .../streaming/examples/NetworkWordCount.scala | 2 +- .../spark/streaming/examples/QueueStream.scala | 2 +- .../spark/streaming/examples/RawNetworkGrep.scala | 2 +- .../examples/StatefulNetworkWordCount.scala | 2 +- .../streaming/examples/TwitterAlgebirdCMS.scala | 2 +- .../streaming/examples/TwitterAlgebirdHLL.scala | 2 +- .../streaming/examples/TwitterPopularTags.scala | 2 +- .../spark/streaming/examples/ZeroMQWordCount.scala | 2 +- .../examples/clickstream/PageViewStream.scala | 2 +- .../apache/spark/streaming/StreamingContext.scala | 6 ++++++ .../streaming/api/java/JavaStreamingContext.scala | 25 +++++++++++++--------- 47 files changed, 75 insertions(+), 54 deletions(-) (limited to 'core/src') 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 5be5317f40..e93b10fd7e 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 @@ -431,4 +431,10 @@ object JavaSparkContext { implicit def fromSparkContext(sc: SparkContext): JavaSparkContext = new JavaSparkContext(sc) implicit def toSparkContext(jsc: JavaSparkContext): SparkContext = jsc.sc + + /** + * Find the JAR from which a given class was loaded, to make it easy for users to pass + * their JARs to SparkContext. + */ + def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index be0d38589c..12f3355bc4 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -106,7 +106,7 @@ public class JavaHdfsLR { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaHdfsLR", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaHdfsLR.class)); JavaRDD lines = sc.textFile(args[1]); JavaRDD points = lines.map(new ParsePoint()).cache(); int ITERATIONS = Integer.parseInt(args[2]); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java index 5a6afe7eae..63465a3bbf 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java @@ -74,7 +74,7 @@ public class JavaKMeans { System.exit(1); } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class)); String path = args[1]; int K = Integer.parseInt(args[2]); double convergeDist = Double.parseDouble(args[3]); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index 407cd7ccfa..74e4d9291a 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -104,7 +104,7 @@ public class JavaLogQuery { } JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaLogQuery.class)); JavaRDD dataSet = (args.length == 2) ? jsc.textFile(args[1]) : jsc.parallelize(exampleApacheLogs); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index 89aed8f279..f774f6a04e 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -17,6 +17,7 @@ package org.apache.spark.examples; +import org.apache.spark.SparkContext; import scala.Tuple2; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -53,7 +54,7 @@ public class JavaPageRank { } JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaPageRank", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaPageRank.class)); // Loads in input file. It should be in format of: // URL neighbor URL diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index 4a2380caf5..5558ab7c03 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -36,7 +36,7 @@ public class JavaSparkPi { } JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaSparkPi.class)); int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2; int n = 100000 * slices; diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 17f21f6b77..99e6ba347c 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -64,7 +64,7 @@ public class JavaTC { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaTC", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaTC.class)); Integer slices = (args.length > 1) ? Integer.parseInt(args[1]): 2; JavaPairRDD tc = sc.parallelizePairs(generateGraph(), slices).cache(); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index bd6383e13d..8a071caf13 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -36,7 +36,7 @@ public class JavaWordCount { } JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaWordCount", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaWordCount.class)); JavaRDD lines = ctx.textFile(args[1], 1); JavaRDD words = lines.flatMap(new FlatMapFunction() { diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java index 45a0d237da..5e1a77baaa 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java @@ -68,7 +68,7 @@ public class JavaALS { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaALS", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaALS.class)); JavaRDD lines = sc.textFile(args[1]); JavaRDD ratings = lines.map(new ParseRating()); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index cd59a139b9..1f12f518a0 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -62,7 +62,7 @@ public class JavaKMeans { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class)); JavaRDD lines = sc.textFile(args[1]); JavaRDD points = lines.map(new ParsePoint()); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index 258061c8e6..593e4df111 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -59,7 +59,7 @@ public class JavaLR { } JavaSparkContext sc = new JavaSparkContext(args[0], "JavaLR", - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaLR.class)); JavaRDD lines = sc.textFile(args[1]); JavaRDD points = lines.map(new ParsePoint()).cache(); double stepSize = Double.parseDouble(args[2]); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java index 261813bf2f..64ac72474b 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java @@ -50,7 +50,8 @@ public class JavaFlumeEventCount { Duration batchInterval = new Duration(2000); JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval, - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), + JavaStreamingContext.jarOfClass(JavaFlumeEventCount.class)); JavaDStream flumeStream = sc.flumeStream("localhost", port); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index 75b588e4b8..0a56e7abdf 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -54,7 +54,8 @@ public class JavaKafkaWordCount { // Create the context with a 1 second batch size JavaStreamingContext ssc = new JavaStreamingContext(args[0], "KafkaWordCount", - new Duration(2000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + new Duration(2000), System.getenv("SPARK_HOME"), + JavaStreamingContext.jarOfClass(JavaKafkaWordCount.class)); int numThreads = Integer.parseInt(args[4]); Map topicMap = new HashMap(); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index def87c199b..ec6f6a8c56 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -48,7 +48,8 @@ public class JavaNetworkWordCount { // Create the context with a 1 second batch size JavaStreamingContext ssc = new JavaStreamingContext(args[0], "NetworkWordCount", - new Duration(1000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + new Duration(1000), System.getenv("SPARK_HOME"), + JavaStreamingContext.jarOfClass(JavaNetworkWordCount.class)); // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited test (eg. generated by 'nc') diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java index c8c7389dd1..4b9fd52713 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java @@ -40,7 +40,7 @@ public class JavaQueueStream { // Create the context JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000), - System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaQueueStream.class)); // Create the queue through which RDDs can be pushed to // a QueueInputDStream diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index a119980992..0097dade19 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -33,7 +33,7 @@ object BroadcastTest { System.setProperty("spark.broadcast.blockSize", blockSize) val sc = new SparkContext(args(0), "Broadcast Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index 92eb96bd8e..b3eb611dd2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -27,7 +27,7 @@ object ExceptionHandlingTest { } val sc = new SparkContext(args(0), "ExceptionHandlingTest", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) sc.parallelize(0 until sc.defaultParallelism).foreach { i => if (math.random > 0.75) throw new Exception("Testing exception handling") diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 42c2e0e8e1..39752fdd0e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -34,7 +34,7 @@ object GroupByTest { var numReducers = if (args.length > 4) args(4).toInt else numMappers val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index efe2e93b0d..65d67356be 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.mapreduce.TableInputFormat object HBaseTest { def main(args: Array[String]) { val sc = new SparkContext(args(0), "HBaseTest", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val conf = HBaseConfiguration.create() diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index d6a88d3032..c3597d94a2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -22,7 +22,7 @@ import org.apache.spark._ object HdfsTest { def main(args: Array[String]) { val sc = new SparkContext(args(0), "HdfsTest", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val file = sc.textFile(args(1)) val mapped = file.map(s => s.length).cache() for (iter <- 1 to 10) { diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index 17ff3ce764..bddb54b39c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -45,7 +45,7 @@ object LogQuery { } val sc = new SparkContext(args(0), "Log Query", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val dataSet = if (args.length == 2) sc.textFile(args(1)) diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index e1afc29f9a..4aef04fc06 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -28,7 +28,7 @@ object MultiBroadcastTest { } val sc = new SparkContext(args(0), "Multi-Broadcast Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 37ddfb5db7..73b0e216ca 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -36,7 +36,7 @@ object SimpleSkewedGroupByTest { var ratio = if (args.length > 5) args(5).toInt else 5.0 val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 9c954b2b5b..31c6d108f3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -34,7 +34,7 @@ object SkewedGroupByTest { var numReducers = if (args.length > 4) args(4).toInt else numMappers val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index 814944ba1c..30c86d83e6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -112,7 +112,7 @@ object SparkALS { printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) val sc = new SparkContext(host, "SparkALS", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val R = generateR() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 86dd9ca1b3..ff72532db1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -54,7 +54,7 @@ object SparkHdfsLR { val inputPath = args(1) val conf = SparkHadoopUtil.get.newConfiguration() val sc = new SparkContext(args(0), "SparkHdfsLR", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")), Map(), + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)))) val lines = sc.textFile(inputPath) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index bc2db39c12..8c99025eaa 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -55,7 +55,7 @@ object SparkKMeans { System.exit(1) } val sc = new SparkContext(args(0), "SparkLocalKMeans", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val lines = sc.textFile(args(1)) val data = lines.map(parseVector _).cache() val K = args(2).toInt diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index 9ed9fe4d76..c54a55bdb4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -49,7 +49,7 @@ object SparkLR { System.exit(1) } val sc = new SparkContext(args(0), "SparkLR", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val numSlices = if (args.length > 1) args(1).toInt else 2 val points = sc.parallelize(generateData, numSlices).cache() 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 a508c0df57..d203f4d20e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -38,7 +38,7 @@ object SparkPageRank { } var iters = args(2).toInt val ctx = new SparkContext(args(0), "PageRank", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val lines = ctx.textFile(args(1), 1) val links = lines.map{ s => val parts = s.split("\\s+") diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index a689e5a360..e5a09ecec0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -29,7 +29,7 @@ object SparkPi { System.exit(1) } val spark = new SparkContext(args(0), "SparkPi", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val slices = if (args.length > 1) args(1).toInt else 2 val n = 100000 * slices val count = spark.parallelize(1 to n, slices).map { i => diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 8543ce0e32..24e8afa26b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -46,7 +46,7 @@ object SparkTC { System.exit(1) } val spark = new SparkContext(args(0), "SparkTC", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) val slices = if (args.length > 1) args(1).toInt else 2 var tc = spark.parallelize(generateGraph, slices).cache() diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 3641517934..546495357f 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -151,7 +151,7 @@ object ActorWordCount { // Create the context and set the batch size val ssc = new StreamingContext(master, "ActorWordCount", Seconds(2), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) /* * Following is the use of actorStream to plug in custom actor as receiver diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala index 9f6e163454..5ef1928294 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala @@ -48,7 +48,7 @@ object FlumeEventCount { val batchInterval = Milliseconds(2000) // Create the context and set the batch size val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval, - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create a flume stream val stream = ssc.flumeStream(host,port,StorageLevel.MEMORY_ONLY) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index 61be1ce4b1..1486d77d8a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -40,7 +40,7 @@ object HdfsWordCount { // Create the context val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create the FileInputDStream on the directory and use the // stream to count words in new files created diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 8dc8a3531a..172091be2e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -48,7 +48,7 @@ object KafkaWordCount { val Array(master, zkQuorum, group, topics, numThreads) = args val ssc = new StreamingContext(master, "KafkaWordCount", Seconds(2), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) ssc.checkpoint("checkpoint") val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index ea138f55e8..2d02ef77c0 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -96,7 +96,7 @@ object MQTTWordCount { val Seq(master, brokerUrl, topic) = args.toSeq val ssc = new StreamingContext(master, "MqttWordCount", Seconds(2), System.getenv("SPARK_HOME"), - Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + StreamingContext.jarOfClass(this.getClass)) val lines = ssc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_ONLY) val words = lines.flatMap(x => x.toString.split(" ")) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala index ce8df8c502..74d76ec26c 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala @@ -41,7 +41,7 @@ object NetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited test (eg. generated by 'nc') 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 fad512eeba..9d640e716b 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 @@ -33,7 +33,7 @@ object QueueStream { // Create the context val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create the queue through which RDDs can be pushed to // a QueueInputDStream 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 0b45c30d20..c0706d0724 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 @@ -49,7 +49,7 @@ object RawNetworkGrep { // Create the context val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Warm up the JVMs on master and slave for JIT compilation to kick in RawTextHelper.warmUp(ssc.sparkContext) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index e55d71edfc..f43c8ab61d 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -49,7 +49,7 @@ object StatefulNetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(1), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) ssc.checkpoint(".") // Create a NetworkInputDStream on target ip:port and count the diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala index 35b6329ab3..9d21d3178f 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -60,7 +60,7 @@ object TwitterAlgebirdCMS { 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"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) 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/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala index 8bfde2a829..5111e6f62a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -49,7 +49,7 @@ object TwitterAlgebirdHLL { 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"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) 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/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala index 27aa6b14bf..7a3df687b7 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala @@ -38,7 +38,7 @@ object TwitterPopularTags { 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"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) val stream = ssc.twitterStream(None, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 2948aa7cc4..89d3042123 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -78,7 +78,7 @@ object ZeroMQWordCount { // Create the context and set the batch size val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index 968b578487..0569846f18 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -42,7 +42,7 @@ object PageViewStream { // Create the context val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) // Create a NetworkInputDStream on target host:port and convert each line to a PageView val pageViews = ssc.socketTextStream(host, port) 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 304986f187..b3a7cf08b9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -595,6 +595,12 @@ object StreamingContext { new PairDStreamFunctions[K, V](stream) } + /** + * Find the JAR from which a given class was loaded, to make it easy for users to pass + * their JARs to SparkContext. + */ + def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls) + protected[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = { // Set the default cleaner delay to an hour if not already set. // This should be sufficient for even 1 second batch intervals. 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 b79173c6aa..7dec4b3ad7 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,29 +17,27 @@ package org.apache.spark.streaming.api.java -import java.lang.{Integer => JInt} import java.io.InputStream -import java.util.{Map => JMap, List => JList} +import java.lang.{Integer => JInt} +import java.util.{List => JList, Map => JMap} import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import akka.actor.{Props, SupervisorStrategy} +import akka.util.ByteString +import akka.zeromq.Subscribe import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import twitter4j.Status -import akka.actor.Props -import akka.actor.SupervisorStrategy -import akka.zeromq.Subscribe -import akka.util.ByteString - import twitter4j.auth.Authorization +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} 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.{JavaPairRDD, JavaSparkContext, JavaRDD} import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream._ -import org.apache.spark.SparkConf import org.apache.spark.streaming.scheduler.StreamingListener /** @@ -716,5 +714,12 @@ class JavaStreamingContext(val ssc: StreamingContext) { * Sstops the execution of the streams. */ def stop() = ssc.stop() +} +object JavaStreamingContext { + /** + * Find the JAR from which a given class was loaded, to make it easy for users to pass + * their JARs to SparkContext. + */ + def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray } -- cgit v1.2.3 From aaaa6731845495743aff4cc9bd64a54b9aa36c27 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 5 Jan 2014 13:57:42 -0800 Subject: Quite akka when remote lifecycle logging is disabled. I noticed when connecting to a standalone cluster Spark gives a bunch of Akka ERROR logs that make it seem like something is failing. This patch does two things: 1. Akka dead letter logging is turned on/off according to the existing lifecycle spark property. 2. We explicitly silence akka's EndpointWriter log in log4j. This is necessary because for some reason that log doesn't pick up on the lifecycle logging settings. After a few hours of debugging this was the only solution I found that worked. --- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 362cea5e3e..5729334e24 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -21,6 +21,8 @@ import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} import com.typesafe.config.ConfigFactory +import org.apache.log4j.{Level, Logger} + import org.apache.spark.SparkConf /** @@ -47,8 +49,13 @@ private[spark] object AkkaUtils { val akkaTimeout = conf.get("spark.akka.timeout", "100").toInt val akkaFrameSize = conf.get("spark.akka.frameSize", "10").toInt - val lifecycleEvents = - if (conf.get("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" + val akkaLogLifecycleEvents = conf.get("spark.akka.logLifecycleEvents", "false").toBoolean + val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" + if (!akkaLogLifecycleEvents) { + Option(Logger.getLogger("akka.remote.EndpointWriter")).map(l => l.setLevel(Level.FATAL)) + } + + val logAkkaConfig = if (conf.get("spark.akka.logAkkaConfig", "false").toBoolean) "on" else "off" val akkaHeartBeatPauses = conf.get("spark.akka.heartbeat.pauses", "600").toInt val akkaFailureDetector = @@ -73,7 +80,10 @@ private[spark] object AkkaUtils { |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}MiB |akka.remote.netty.tcp.execution-pool-size = $akkaThreads |akka.actor.default-dispatcher.throughput = $akkaBatchSize + |akka.log-config-on-start = $logAkkaConfig |akka.remote.log-remote-lifecycle-events = $lifecycleEvents + |akka.log-dead-letters = $lifecycleEvents + |akka.log-dead-letters-during-shutdown = $lifecycleEvents """.stripMargin) val actorSystem = if (indestructible) { -- cgit v1.2.3 From 94fdcda89638498f127abf3bb5231064182b4945 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 5 Jan 2014 15:10:05 -0800 Subject: Provide logging when attempts to connect to the master fail. Without these it's a bit less clear what's going on for the user. One thing I realize when doing this is that akka itself actually retries the initial association. So the retry we currently have is redundant with akka's. --- .../main/scala/org/apache/spark/deploy/client/Client.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 9bbd635ab9..481026eaa2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -24,7 +24,8 @@ import scala.concurrent.duration._ import akka.actor._ import akka.pattern.ask -import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent} + import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ @@ -110,6 +111,12 @@ private[spark] class Client( } } + private def isPossibleMaster(remoteUrl: Address) = { + masterUrls.map(s => Master.toAkkaUrl(s)) + .map(u => AddressFromURIString(u).hostPort) + .contains(remoteUrl.hostPort) + } + override def receive = { case RegisteredApplication(appId_, masterUrl) => appId = appId_ @@ -145,6 +152,9 @@ private[spark] class Client( logWarning(s"Connection to $address failed; waiting for master to reconnect...") markDisconnected() + case AssociationErrorEvent(cause, _, address, _) if isPossibleMaster(address) => + logWarning(s"Could not connect to $address: $cause") + case StopClient => markDead() sender ! true -- cgit v1.2.3 From 63f906322d173c0e5e74c815d638db1bea338340 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 5 Jan 2014 15:52:43 -0800 Subject: Fall back to zero-arg constructor for Serializer initialization if there is no constructor that accepts SparkConf. This maintains backward compatibility with older serializers implemented by users. --- .../scala/org/apache/spark/serializer/Serializer.scala | 3 +++ .../org/apache/spark/serializer/SerializerManager.scala | 15 +++++++++++++-- 2 files changed, 16 insertions(+), 2 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index 160cca4d6c..9a5e3cb77e 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -29,6 +29,9 @@ import org.apache.spark.util.{NextIterator, ByteBufferInputStream} * A serializer. Because some serialization libraries are not thread safe, this class is used to * 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. + * + * Implementations of this trait should have a zero-arg constructor or a constructor that accepts a + * [[org.apache.spark.SparkConf]] as parameter. If both constructors are defined, the latter takes precedence. */ trait Serializer { def newInstance(): SerializerInstance diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index 22465272f3..36a37af4f8 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -27,6 +27,7 @@ import org.apache.spark.SparkConf * creating a new one. */ private[spark] class SerializerManager { + // TODO: Consider moving this into SparkConf itself to remove the global singleton. private val serializers = new ConcurrentHashMap[String, Serializer] private var _default: Serializer = _ @@ -53,8 +54,18 @@ private[spark] class SerializerManager { if (serializer == null) { val clsLoader = Thread.currentThread.getContextClassLoader val cls = Class.forName(clsName, true, clsLoader) - val constructor = cls.getConstructor(classOf[SparkConf]) - serializer = constructor.newInstance(conf).asInstanceOf[Serializer] + + // First try with the constructor that takes SparkConf. If we can't find one, + // use a no-arg constructor instead. + try { + val constructor = cls.getConstructor(classOf[SparkConf]) + serializer = constructor.newInstance(conf).asInstanceOf[Serializer] + } catch { + case _: NoSuchMethodException => + val constructor = cls.getConstructor() + serializer = constructor.newInstance().asInstanceOf[Serializer] + } + serializers.put(clsName, serializer) } serializer -- cgit v1.2.3 From a4048ff31e6f8d3e1451d8ae2d5b9edee42cfbbe Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Mon, 6 Jan 2014 09:18:17 +0800 Subject: Get rid of `Either[ActorRef, ActorSelection]' Although we can send messages via an ActorSelection, it would be better to identify the actor and obtain an ActorRef first, so that we can get informed earlier if the remote actor doesn't exist, and get rid of the annoying Either wrapper. --- .../main/scala/org/apache/spark/MapOutputTracker.scala | 14 ++------------ core/src/main/scala/org/apache/spark/SparkEnv.scala | 16 ++++++++-------- .../org/apache/spark/storage/BlockManagerMaster.scala | 8 ++------ .../scala/org/apache/spark/storage/ThreadingTest.scala | 2 +- .../src/main/scala/org/apache/spark/util/AkkaUtils.scala | 5 +++++ .../org/apache/spark/storage/DiskBlockManagerSuite.scala | 4 +--- 6 files changed, 19 insertions(+), 30 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index cdae167aef..77b8ca1cce 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -55,7 +55,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { private val timeout = AkkaUtils.askTimeout(conf) // Set to the MapOutputTrackerActor living on the driver - var trackerActor: Either[ActorRef, ActorSelection] = _ + var trackerActor: ActorRef = _ protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] @@ -71,17 +71,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { // throw a SparkException if this fails. private def askTracker(message: Any): Any = { try { - /* - The difference between ActorRef and ActorSelection is well explained here: - http://doc.akka.io/docs/akka/2.2.3/project/migration-guide-2.1.x-2.2.x.html#Use_actorSelection_instead_of_actorFor - In spark a map output tracker can be either started on Driver where it is created which - is an ActorRef or it can be on executor from where it is looked up which is an - actorSelection. - */ - val future = trackerActor match { - case Left(a: ActorRef) => a.ask(message)(timeout) - case Right(b: ActorSelection) => b.ask(message)(timeout) - } + val future = trackerActor.ask(message)(timeout) Await.result(future, timeout) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 634a94f0a7..2e36ccb9a0 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -17,11 +17,10 @@ package org.apache.spark -import collection.mutable -import serializer.Serializer +import scala.collection.mutable +import scala.concurrent.Await import akka.actor._ -import akka.remote.RemoteActorRefProvider import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem @@ -157,17 +156,18 @@ object SparkEnv extends Logging { conf.get("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"), conf) - def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = { + def registerOrLookup(name: String, newActor: => Actor): ActorRef = { if (isDriver) { logInfo("Registering " + name) - Left(actorSystem.actorOf(Props(newActor), name = name)) + actorSystem.actorOf(Props(newActor), name = name) } else { val driverHost: String = conf.get("spark.driver.host", "localhost") val driverPort: Int = conf.get("spark.driver.port", "7077").toInt Utils.checkHost(driverHost, "Expected hostname") - val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name) - logInfo("Connecting to " + name + ": " + url) - Right(actorSystem.actorSelection(url)) + val url = s"akka.tcp://spark@$driverHost:$driverPort/user/$name" + val timeout = AkkaUtils.lookupTimeout(conf) + logInfo(s"Connecting to $name: $url") + Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index b5afe8cd23..51a29ed8ef 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -28,8 +28,7 @@ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.AkkaUtils private[spark] -class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection], - conf: SparkConf) extends Logging { +class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Logging { val AKKA_RETRY_ATTEMPTS: Int = conf.get("spark.akka.num.retries", "3").toInt val AKKA_RETRY_INTERVAL_MS: Int = conf.get("spark.akka.retry.wait", "3000").toInt @@ -159,10 +158,7 @@ class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection], while (attempts < AKKA_RETRY_ATTEMPTS) { attempts += 1 try { - val future = driverActor match { - case Left(a: ActorRef) => a.ask(message)(timeout) - case Right(b: ActorSelection) => b.ask(message)(timeout) - } + val future = driverActor.ask(message)(timeout) val result = Await.result(future, timeout) if (result == null) { throw new SparkException("BlockManagerMaster returned null") 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 dca98c6c05..729ba2c550 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -95,7 +95,7 @@ private[spark] object ThreadingTest { val conf = new SparkConf() val serializer = new KryoSerializer(conf) val blockManagerMaster = new BlockManagerMaster( - Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf) + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 362cea5e3e..b4c4e1dbbc 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -91,4 +91,9 @@ private[spark] object AkkaUtils { def askTimeout(conf: SparkConf): FiniteDuration = { Duration.create(conf.get("spark.akka.askTimeout", "30").toLong, "seconds") } + + /** Returns the default Spark timeout to use for Akka remote actor lookup. */ + def lookupTimeout(conf: SparkConf): FiniteDuration = { + Duration.create(conf.get("spark.akka.lookupTimeout", "30").toLong, "seconds") + } } diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index af4b31d53c..829f389460 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -23,9 +23,7 @@ import scala.collection.mutable import com.google.common.io.Files import org.apache.spark.SparkConf -import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} -import scala.util.Try -import akka.actor.{Props, ActorSelection, ActorSystem} +import org.scalatest.{BeforeAndAfterEach, FunSuite} class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { private val testConf = new SparkConf(false) -- cgit v1.2.3 From 5c152e3e219a44f97d9df38ba00cdc4adbf4d873 Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Mon, 6 Jan 2014 10:39:05 +0800 Subject: Fixed several compilation errors in test suites --- .../test/scala/org/apache/spark/MapOutputTrackerSuite.scala | 11 +++++++---- .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 2 +- 2 files changed, 8 insertions(+), 5 deletions(-) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 10b8b441fd..82dc30ecc4 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -23,6 +23,7 @@ import akka.actor._ import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AkkaUtils +import scala.concurrent.Await class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { private val conf = new SparkConf @@ -101,13 +102,15 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { System.setProperty("spark.hostPort", hostname + ":" + boundPort) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = Left(actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker")) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf) val slaveTracker = new MapOutputTracker(conf) - slaveTracker.trackerActor = Right(slaveSystem.actorSelection( - "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker")) + val selection = slaveSystem.actorSelection( + s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + val timeout = AkkaUtils.lookupTimeout(conf) + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) masterTracker.registerShuffle(10, 1) masterTracker.incrementEpoch() 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 a0fc3445be..032c2f2f69 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -58,7 +58,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT conf.set("spark.hostPort", "localhost:" + boundPort) master = new BlockManagerMaster( - Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf) + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case System.setProperty("os.arch", "amd64") -- cgit v1.2.3 From eb24684748da5dc2495fc4afe6da58edb463294b Mon Sep 17 00:00:00 2001 From: "Lian, Cheng" Date: Mon, 6 Jan 2014 11:21:35 +0800 Subject: Fixed test suite compilation errors --- core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 82dc30ecc4..afc1beff98 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -50,14 +50,14 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master start and stop") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))) tracker.stop() } test("master register and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -76,7 +76,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master register and unregister and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))) + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) -- cgit v1.2.3 From 675d7eb4f064129d275a45df4c5c43f558638422 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 5 Jan 2014 21:23:14 -0800 Subject: Responding to Aaron's review --- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 2 ++ 1 file changed, 2 insertions(+) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 5729334e24..7df7e3d8e5 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -52,6 +52,8 @@ private[spark] object AkkaUtils { val akkaLogLifecycleEvents = conf.get("spark.akka.logLifecycleEvents", "false").toBoolean val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" if (!akkaLogLifecycleEvents) { + // As a workaround for Akka issue #3787, we coerce the "EndpointWriter" log to be silent. + // See: https://www.assembla.com/spaces/akka/tickets/3787#/ Option(Logger.getLogger("akka.remote.EndpointWriter")).map(l => l.setLevel(Level.FATAL)) } -- cgit v1.2.3 From 25446dd931cce5916de5dddf4689b41ee6fd3148 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 6 Jan 2014 07:58:59 -0600 Subject: Add warning to null setJars check --- core/src/main/scala/org/apache/spark/SparkConf.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 7073a999a9..55f27033b5 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -24,7 +24,7 @@ import com.typesafe.config.ConfigFactory * * @param loadDefaults whether to load values from the system properties and classpath */ -class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { +class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable with Logging { /** Create a SparkConf that loads defaults from system properties and the classpath */ def this() = this(true) @@ -67,6 +67,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable { /** Set JAR files to distribute to the cluster. */ def setJars(jars: Seq[String]): SparkConf = { + for (jar <- jars if (jar == null)) logWarning("null jar passed to SparkContext constructor") set("spark.jars", jars.filter(_ != null).mkString(",")) } -- cgit v1.2.3 From 9272a004af7dd464809d9effc4ddc248a53919df Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 6 Jan 2014 13:03:19 -0800 Subject: Fix test breaking downstream builds --- .../test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index f58b1ee05a..7e5aaa3f98 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -26,7 +26,7 @@ import org.apache.spark.deploy.{ExecutorState, Command, ApplicationDescription} class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) - val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.env.get("spark.home")).get + val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get val appDesc = new ApplicationDescription("app name", 8, 500, Command("foo", Seq(),Map()), sparkHome, "appUiUrl") val appId = "12345-worker321-9876" -- cgit v1.2.3 From b3018811e106e6414816380a35c07a8564945d37 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 6 Jan 2014 15:47:40 +0530 Subject: Allow users to set arbitrary akka configurations via spark conf. --- core/src/main/scala/org/apache/spark/SparkConf.scala | 7 +++++++ core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 8 +++++--- docs/configuration.md | 8 ++++++++ 3 files changed, 20 insertions(+), 3 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 55f27033b5..2d437f1b21 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -172,6 +172,13 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable with .map{case (k, v) => (k.substring(prefix.length), v)} } + /** Get all akka conf variables set on this SparkConf */ + def getAkkaConf: Seq[(String, String)] = { + getAll.filter { + case (k, v) => k.startsWith("akka.") + } + } + /** Does the configuration contain a given parameter? */ def contains(key: String): Boolean = settings.contains(key) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 7df7e3d8e5..2ee37815de 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -17,12 +17,13 @@ package org.apache.spark.util +import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} import com.typesafe.config.ConfigFactory -import org.apache.log4j.{Level, Logger} +import org.apache.log4j.{Level, Logger} import org.apache.spark.SparkConf /** @@ -64,7 +65,8 @@ private[spark] object AkkaUtils { conf.get("spark.akka.failure-detector.threshold", "300.0").toDouble val akkaHeartBeatInterval = conf.get("spark.akka.heartbeat.interval", "1000").toInt - val akkaConf = ConfigFactory.parseString( + val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]).withFallback( + ConfigFactory.parseString( s""" |akka.daemonic = on |akka.loggers = [""akka.event.slf4j.Slf4jLogger""] @@ -86,7 +88,7 @@ private[spark] object AkkaUtils { |akka.remote.log-remote-lifecycle-events = $lifecycleEvents |akka.log-dead-letters = $lifecycleEvents |akka.log-dead-letters-during-shutdown = $lifecycleEvents - """.stripMargin) + """.stripMargin)) val actorSystem = if (indestructible) { IndestructibleActorSystem(name, akkaConf) diff --git a/docs/configuration.md b/docs/configuration.md index 09342fedfc..8a8857bb3b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -360,6 +360,14 @@ Apart from these, the following properties are also available, and may be useful Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, BlockManager might take a performance hit. + + akka.x.y.... + value + + An arbitrary akka configuration can be set directly on spark conf and it is applied for all the ActorSystems created spark wide for that spark context and its assigned executors as well. + + + spark.shuffle.consolidateFiles false -- cgit v1.2.3 From b84dc780d3ad3fc2f0f1e9ba3fd1676447c65e96 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 6 Jan 2014 15:51:32 +0530 Subject: Allow configuration to be printed in logs for diagnosis. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 ++++ docs/configuration.md | 7 +++++++ 2 files changed, 11 insertions(+) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e80e43af6d..99dcced7d7 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -116,6 +116,10 @@ class SparkContext( throw new SparkException("An application must be set in your configuration") } + if (conf.get("spark.log-conf", "false").toBoolean) { + logInfo("Spark configuration:\n" + conf.toDebugString) + } + // Set Spark driver host and port system properties conf.setIfMissing("spark.driver.host", Utils.localHostName()) conf.setIfMissing("spark.driver.port", "0") diff --git a/docs/configuration.md b/docs/configuration.md index 8a8857bb3b..f00fe05476 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -403,6 +403,13 @@ Apart from these, the following properties are also available, and may be useful How many times slower a task is than the median to be considered for speculation. + + spark.log-conf + false + + Log the supplied SparkConf as INFO at start of spark context. + + ## Viewing Spark Properties -- cgit v1.2.3 From c729fa7c8ed733a778a7201ed17bf74f3e132845 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 7 Jan 2014 13:08:16 +0530 Subject: formatting related fixes suggested by Patrick. --- core/src/main/scala/org/apache/spark/SparkConf.scala | 6 +----- docs/configuration.md | 2 +- 2 files changed, 2 insertions(+), 6 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 2d437f1b21..b166527614 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -173,11 +173,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable with } /** Get all akka conf variables set on this SparkConf */ - def getAkkaConf: Seq[(String, String)] = { - getAll.filter { - case (k, v) => k.startsWith("akka.") - } - } + def getAkkaConf: Seq[(String, String)] = getAll.filter {case (k, v) => k.startsWith("akka.")} /** Does the configuration contain a given parameter? */ def contains(key: String): Boolean = settings.contains(key) diff --git a/docs/configuration.md b/docs/configuration.md index f00fe05476..1d36ecb9c1 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -364,7 +364,7 @@ Apart from these, the following properties are also available, and may be useful akka.x.y.... value - An arbitrary akka configuration can be set directly on spark conf and it is applied for all the ActorSystems created spark wide for that spark context and its assigned executors as well. + An arbitrary akka configuration can be set directly on spark conf and it is applied for all the ActorSystems created spark wide for that SparkContext and its assigned executors as well. -- cgit v1.2.3 From d8bcc8e9a095c1b20dd7a17b6535800d39bff80e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 7 Jan 2014 14:35:52 -0500 Subject: Add way to limit default # of cores used by applications on standalone mode Also documents the spark.deploy.spreadOut option. --- .../main/scala/org/apache/spark/SparkConf.scala | 7 ++++- .../main/scala/org/apache/spark/SparkContext.scala | 2 +- .../spark/deploy/master/ApplicationInfo.scala | 7 +++-- .../org/apache/spark/deploy/master/Master.scala | 8 ++++-- docs/configuration.md | 33 +++++++++++++++++++--- docs/css/bootstrap.min.css | 2 +- docs/job-scheduling.md | 5 ++-- docs/spark-standalone.md | 10 +++++++ 8 files changed, 60 insertions(+), 14 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index b166527614..2de32231e8 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -67,7 +67,7 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable with /** Set JAR files to distribute to the cluster. */ def setJars(jars: Seq[String]): SparkConf = { - for (jar <- jars if (jar == null)) logWarning("null jar passed to SparkContext constructor") + for (jar <- jars if (jar == null)) logWarning("null jar passed to SparkContext constructor") set("spark.jars", jars.filter(_ != null).mkString(",")) } @@ -165,6 +165,11 @@ class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable with getOption(key).map(_.toDouble).getOrElse(defaultValue) } + /** Get a parameter as a boolean, falling back to a default if not set */ + def getBoolean(key: String, defaultValue: Boolean): Boolean = { + getOption(key).map(_.toBoolean).getOrElse(defaultValue) + } + /** Get all executor environment variables set on this SparkConf */ def getExecutorEnv: Seq[(String, String)] = { val prefix = "spark.executorEnv." diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 99dcced7d7..0e47f4e442 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -116,7 +116,7 @@ class SparkContext( throw new SparkException("An application must be set in your configuration") } - if (conf.get("spark.log-conf", "false").toBoolean) { + if (conf.get("spark.logConf", "false").toBoolean) { logInfo("Spark configuration:\n" + conf.toDebugString) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 5150b7c7de..1321d9200b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -28,7 +28,8 @@ private[spark] class ApplicationInfo( val desc: ApplicationDescription, val submitDate: Date, val driver: ActorRef, - val appUiUrl: String) + val appUiUrl: String, + defaultCores: Int) extends Serializable { @transient var state: ApplicationState.Value = _ @@ -81,7 +82,9 @@ private[spark] class ApplicationInfo( } } - def coresLeft: Int = desc.maxCores - coresGranted + private val myMaxCores = if (desc.maxCores == Int.MaxValue) defaultCores else desc.maxCores + + def coresLeft: Int = myMaxCores - coresGranted private var _retryCount = 0 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 7b696cfcca..ee01fb11df 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 @@ -88,7 +88,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - val spreadOutApps = conf.get("spark.deploy.spreadOut", "true").toBoolean + val spreadOutApps = conf.getBoolean("spark.deploy.spreadOut", true) + + // Default maxCores for applications that don't specify it (i.e. pass Int.MaxValue) + val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue) override def preStart() { logInfo("Starting Spark master at " + masterUrl) @@ -426,7 +429,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) - new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl) + new ApplicationInfo( + now, newApplicationId(date), desc, date, driver, desc.appUiUrl, defaultCores) } def registerApplication(app: ApplicationInfo): Unit = { diff --git a/docs/configuration.md b/docs/configuration.md index 1d36ecb9c1..52ed59be30 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -77,13 +77,14 @@ there are at least five properties that you will commonly want to control: spark.cores.max - (infinite) + (not set) When running on a standalone deploy cluster or a Mesos cluster in "coarse-grained" sharing mode, the maximum amount of CPU cores to request for the application from - across the cluster (not from each machine). The default will use all available cores - offered by the cluster manager. + across the cluster (not from each machine). If not set, the default will be + spark.deploy.defaultCores on Spark's standalone cluster manager, or + infinite (all available cores) on Mesos. @@ -404,12 +405,36 @@ Apart from these, the following properties are also available, and may be useful - spark.log-conf + spark.logConf false Log the supplied SparkConf as INFO at start of spark context. + + spark.deploy.spreadOut + true + + Whether the standalone cluster manager should spread applications out across nodes or try + to consolidate them onto as few nodes as possible. Spreading out is usually better for + data locality in HDFS, but consolidating is more efficient for compute-intensive workloads.
    + Note: this setting needs to be configured in the cluster master, not in individual + applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. + + + + spark.deploy.defaultCores + (infinite) + + Default number of cores to give to applications in Spark's standalone mode if they don't + set spark.cores.max. If not set, applications always get all available + cores unless they configure spark.cores.max themselves. + Set this lower on a shared cluster to prevent users from grabbing + the whole cluster by default.
    + Note: this setting needs to be configured in the cluster master, not in individual + applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. + + ## Viewing Spark Properties diff --git a/docs/css/bootstrap.min.css b/docs/css/bootstrap.min.css index 119bb5dba7..30ea440c00 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,#feecda);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#feecda));background-image:-webkit-linear-gradient(top,#fff,#feecda);background-image:-o-linear-gradient(top,#fff,#feecda);background-image:linear-gradient(to bottom,#fff,#feecda);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='#fffeecda',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,#feead8);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#feead8));background-image:-webkit-linear-gradient(top,#fff,#feead8);background-image:-o-linear-gradient(top,#fff,#feead8);background-image:linear-gradient(to bottom,#fff,#feead8);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='#fffeead8',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/job-scheduling.md b/docs/job-scheduling.md index 5951155fe3..df2faa5e41 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -32,9 +32,8 @@ Resource allocation can be configured as follows, based on the cluster type: * **Standalone mode:** By default, applications submitted to the standalone mode cluster will run in FIFO (first-in-first-out) order, and each application will try to use all available nodes. You can limit - the number of nodes an application uses by setting the `spark.cores.max` configuration property in it. This - will allow multiple users/applications to run concurrently. For example, you might launch a long-running - server that uses 10 cores, and allow users to launch shells that use 20 cores each. + the number of nodes an application uses by setting the `spark.cores.max` configuration property in it, + or change the default for applications that don't set this setting through `spark.deploy.defaultCores`. Finally, in addition to controlling cores, each application's `spark.executor.memory` setting controls its memory use. * **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` configuration property to `true`, diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index c851833a18..f47d41f966 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -167,6 +167,16 @@ val conf = new SparkConf() val sc = new SparkContext(conf) {% endhighlight %} +In addition, you can configure `spark.deploy.defaultCores` on the cluster master process to change the +default for applications that don't set `spark.cores.max` to something less than infinite. +Do this by adding the following to `conf/spark-env.sh`: + +{% highlight bash %} +export SPARK_JAVA_OPTS="-Dspark.deploy.defaultCores=" +{% endhighlight %} + +This is useful on shared clusters where users might not have configured a maximum number of cores +individually. # Monitoring and Logging -- cgit v1.2.3 From e688e11206401850a13a87d7db52941cc716f88a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 7 Jan 2014 12:42:19 -0800 Subject: Add log4j exclusion rule to maven. To make this work I had to rename the defaults file. Otherwise maven's pattern matching rules included it when trying to match other log4j.properties files. I also fixed a bug in the existing maven build where two tags were present in assembly/pom.xml such that one overwrote the other. --- assembly/pom.xml | 6 +++--- .../main/resources/org/apache/spark/default-log4j.properties | 8 -------- .../main/resources/org/apache/spark/log4j-defaults.properties | 8 ++++++++ core/src/main/scala/org/apache/spark/Logging.scala | 10 ++++++---- examples/pom.xml | 3 +++ 5 files changed, 20 insertions(+), 15 deletions(-) delete mode 100644 core/src/main/resources/org/apache/spark/default-log4j.properties create mode 100644 core/src/main/resources/org/apache/spark/log4j-defaults.properties (limited to 'core/src') diff --git a/assembly/pom.xml b/assembly/pom.xml index 9b70812c64..54a25910ce 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -108,12 +108,12 @@ META-INF/services/org.apache.hadoop.fs.FileSystem - - - reference.conf + + log4j.properties + diff --git a/core/src/main/resources/org/apache/spark/default-log4j.properties b/core/src/main/resources/org/apache/spark/default-log4j.properties deleted file mode 100644 index d72dbadc39..0000000000 --- a/core/src/main/resources/org/apache/spark/default-log4j.properties +++ /dev/null @@ -1,8 +0,0 @@ -# Set everything to be logged to the console -log4j.rootCategory=INFO, console -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %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/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties new file mode 100644 index 0000000000..d72dbadc39 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -0,0 +1,8 @@ +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %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/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index d519fc5a29..4a34989e50 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -104,13 +104,15 @@ trait Logging { // If Log4j doesn't seem initialized, load a default properties file val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements if (!log4jInitialized) { - val defaultLogProps = "org/apache/spark/default-log4j.properties" + val defaultLogProps = "org/apache/spark/log4j-defaults.properties" val classLoader = this.getClass.getClassLoader Option(classLoader.getResource(defaultLogProps)) match { - case Some(url) => PropertyConfigurator.configure(url) - case None => System.err.println(s"Spark was unable to load $defaultLogProps") + case Some(url) => + PropertyConfigurator.configure(url) + log.info(s"Using Spark's default log4j profile: $defaultLogProps") + case None => + System.err.println(s"Spark was unable to load $defaultLogProps") } - log.info(s"Using Spark's default log4j profile: $defaultLogProps") } Logging.initialized = true diff --git a/examples/pom.xml b/examples/pom.xml index 7a7032c319..7e41bef252 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -203,6 +203,9 @@ reference.conf + + log4j.properties + -- cgit v1.2.3 From 044c8ad3a47d245198a16a68e36a417e80e8c37e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 7 Jan 2014 16:12:20 -0500 Subject: Fix unit test compilation --- core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 0b38e239f9..810ebf4140 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -73,7 +73,8 @@ class JsonProtocolSuite extends FunSuite { new ApplicationDescription("name", 4, 1234, cmd, "sparkHome", "appUiUrl") } def createAppInfo() : ApplicationInfo = { - new ApplicationInfo(3, "id", createAppDesc(), new Date(123456789), null, "appUriStr") + new ApplicationInfo( + 3, "id", createAppDesc(), new Date(123456789), null, "appUriStr", Int.MaxValue) } def createWorkerInfo() : WorkerInfo = { new WorkerInfo("id", "host", 8080, 4, 1234, null, 80, "publicAddress") -- cgit v1.2.3 From 2c421749eae1e3945ca34ce006addd98a0c1a00b Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 7 Jan 2014 19:30:23 -0500 Subject: Address review comments --- .../main/scala/org/apache/spark/deploy/ApplicationDescription.scala | 2 +- core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala | 2 +- .../main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala | 2 +- core/src/main/scala/org/apache/spark/deploy/master/Master.scala | 3 +++ .../apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala | 2 +- core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala | 2 +- .../scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala | 2 +- docs/configuration.md | 4 ++-- 8 files changed, 11 insertions(+), 8 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 19d393a0db..e38459b883 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy private[spark] class ApplicationDescription( val name: String, - val maxCores: Int, /* Integer.MAX_VALUE denotes an unlimited number of cores */ + val maxCores: Option[Int], val memoryPerSlave: Int, val command: Command, val sparkHome: String, 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 ef649fd80c..28ebbdc66b 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 @@ -48,7 +48,7 @@ private[spark] object TestClient { val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, conf = new SparkConf) val desc = new ApplicationDescription( - "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), + "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored") val listener = new TestListener val client = new Client(actorSystem, Array(url), desc, listener, new SparkConf) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 1321d9200b..3e26379166 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -82,7 +82,7 @@ private[spark] class ApplicationInfo( } } - private val myMaxCores = if (desc.maxCores == Int.MaxValue) defaultCores else desc.maxCores + private val myMaxCores = desc.maxCores.getOrElse(defaultCores) def coresLeft: Int = myMaxCores - coresGranted 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 ee01fb11df..6617b7100f 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 @@ -92,6 +92,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // Default maxCores for applications that don't specify it (i.e. pass Int.MaxValue) val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue) + if (defaultCores < 1) { + throw new SparkException("spark.deploy.defaultCores must be positive") + } override def preStart() { logInfo("Starting Spark master at " + masterUrl) 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 9858717d13..73fc37444e 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 @@ -38,7 +38,7 @@ private[spark] class SparkDeploySchedulerBackend( var stopping = false var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ - val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt + val maxCores = conf.getOption("spark.cores.max").map(_.toInt) override def start() { super.start() diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 810ebf4140..331fa3a642 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -70,7 +70,7 @@ class JsonProtocolSuite extends FunSuite { def createAppDesc() : ApplicationDescription = { val cmd = new Command("mainClass", List("arg1", "arg2"), Map()) - new ApplicationDescription("name", 4, 1234, cmd, "sparkHome", "appUiUrl") + new ApplicationDescription("name", Some(4), 1234, cmd, "sparkHome", "appUiUrl") } def createAppInfo() : ApplicationInfo = { new ApplicationInfo( diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 7e5aaa3f98..be93074b7b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -27,7 +27,7 @@ class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get - val appDesc = new ApplicationDescription("app name", 8, 500, Command("foo", Seq(),Map()), + val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(),Map()), sparkHome, "appUiUrl") val appId = "12345-worker321-9876" val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome), diff --git a/docs/configuration.md b/docs/configuration.md index 52ed59be30..1d6c3d1633 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -418,7 +418,7 @@ Apart from these, the following properties are also available, and may be useful Whether the standalone cluster manager should spread applications out across nodes or try to consolidate them onto as few nodes as possible. Spreading out is usually better for data locality in HDFS, but consolidating is more efficient for compute-intensive workloads.
    - Note: this setting needs to be configured in the cluster master, not in individual + Note: this setting needs to be configured in the standalone cluster master, not in individual applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. @@ -431,7 +431,7 @@ Apart from these, the following properties are also available, and may be useful cores unless they configure spark.cores.max themselves. Set this lower on a shared cluster to prevent users from grabbing the whole cluster by default.
    - Note: this setting needs to be configured in the cluster master, not in individual + Note: this setting needs to be configured in the standalone cluster master, not in individual applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. -- cgit v1.2.3 From 86ed1ad2520662f4a16e535cc05bf2296e6053df Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 7 Jan 2014 16:39:37 -0800 Subject: Fix BlockManagerSuite#after --- .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) (limited to 'core/src') 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 a0fc3445be..fded582640 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -40,8 +40,6 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT var actorSystem: ActorSystem = null var master: BlockManagerMaster = null var oldArch: String = null - var oldOops: String = null - var oldHeartBeat: String = null // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test conf.set("spark.kryoserializer.buffer.mb", "1") @@ -61,7 +59,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - System.setProperty("os.arch", "amd64") + oldArch = System.setProperty("os.arch", "amd64") conf.set("os.arch", "amd64") conf.set("spark.test.useCompressedOops", "true") conf.set("spark.storage.disableBlockManagerHeartBeat", "true") @@ -94,11 +92,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT System.clearProperty("os.arch") } - if (oldOops != null) { - conf.set("spark.test.useCompressedOops", oldOops) - } else { - System.clearProperty("spark.test.useCompressedOops") - } + System.clearProperty("spark.test.useCompressedOops") } test("StorageLevel object caching") { -- cgit v1.2.3 From 4517326ec68d15f7d9bedade5f9bba33d760e6b4 Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Tue, 7 Jan 2014 22:55:56 -0800 Subject: Remove calls to deprecated mapred's OutputCommitter.cleanupJob because since Hadoop 1.0.4 the mapred OutputCommitter.commitJob should do cleanup job. In fact the implementation of mapred OutputCommitter.commitJob looks like this: public void commitJob(JobContext jobContext) throws IOException { cleanupJob(jobContext); } (The jobContext input argument is type of org.apache.hadoop.mapred.JobContext) --- core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala | 4 ---- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 -- 2 files changed, 6 deletions(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 103a1c2051..618d95015f 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -127,10 +127,6 @@ class SparkHadoopWriter(@transient jobConf: JobConf) cmtr.commitJob(getJobContext()) } - def cleanup() { - getOutputCommitter().cleanupJob(getJobContext()) - } - // ********* Private Functions ********* private def getOutputFormat(): OutputFormat[AnyRef,AnyRef] = { diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 04a8d05988..629fb390af 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -638,7 +638,6 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) jobCommitter.setupJob(jobTaskContext) val count = self.context.runJob(self, writeShard _).sum jobCommitter.commitJob(jobTaskContext) - jobCommitter.cleanupJob(jobTaskContext) } /** @@ -728,7 +727,6 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) self.context.runJob(self, writeToFile _) writer.commitJob() - writer.cleanup() } /** -- cgit v1.2.3 From f6b6f88367351f99d02a7de0dbd5c1980cc97bbf Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Tue, 7 Jan 2014 23:23:17 -0800 Subject: Set boolean param name for two files call to SparkHadoopMapReduceUtil.newTaskAttemptID to make it clear which param being set. --- core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) (limited to 'core/src') 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 2662d48c84..73d15b9082 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -76,7 +76,7 @@ class NewHadoopRDD[K, V]( val split = theSplit.asInstanceOf[NewHadoopPartition] logInfo("Input split: " + split.serializableHadoopSplit) val conf = confBroadcast.value.value - val attemptId = newTaskAttemptID(jobtrackerId, id, true, split.index, 0) + val attemptId = newTaskAttemptID(jobtrackerId, id, isMap = true, split.index, 0) val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) val format = inputFormatClass.newInstance if (format.isInstanceOf[Configurable]) { diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 04a8d05988..c8446fd0fc 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -613,7 +613,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt /* "reduce task" */ - val attemptId = newTaskAttemptID(jobtrackerID, stageId, false, context.partitionId, attemptNumber) + val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = outputFormatClass.newInstance val committer = format.getOutputCommitter(hadoopContext) @@ -632,7 +632,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * however we're only going to use this local OutputCommitter for * setupJob/commitJob, so we just use a dummy "map" task. */ - val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, true, 0, 0) + val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = true, 0, 0) val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId) val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext) jobCommitter.setupJob(jobTaskContext) -- cgit v1.2.3 From aa56585d2148b3ced506d2fff89da0858300928c Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Wed, 8 Jan 2014 00:38:29 -0800 Subject: Resolve PR review over 100 chars --- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) (limited to 'core/src') diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index c8446fd0fc..4fe3bc552b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -613,7 +613,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt /* "reduce task" */ - val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, attemptNumber) + val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, + attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = outputFormatClass.newInstance val committer = format.getOutputCommitter(hadoopContext) -- cgit v1.2.3