");
59 | $('#jarsTable > tbody:last').append(items.join(""));
60 | });
61 | });
62 | }
63 |
64 | $(document).ready(getJobs());
65 | $(document).ready(getContexts());
66 | $(document).ready(getJars());
67 |
68 | $(function () {
69 | $('#navTabs a[data-toggle="tab"]').on('show.bs.tab', function (e) {
70 | var target = $(e.target).attr("href");
71 |
72 | if (target == '#jobs') {
73 | getJobs();
74 | } else if (target == "#contexts") {
75 | getContexts();
76 | } else {
77 | getJars();
78 | }
79 | })
80 | });
81 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/CommonMessages.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.ActorRef
4 | import org.joda.time.DateTime
5 |
6 | trait StatusMessage {
7 | val jobId: String
8 | }
9 |
10 | // Messages that are sent and received by multiple actors.
11 | object CommonMessages {
12 | // job status messages
13 | case class JobStarted(jobId: String, context: String, startTime: DateTime) extends StatusMessage
14 | case class JobFinished(jobId: String, endTime: DateTime) extends StatusMessage
15 | case class JobValidationFailed(jobId: String, endTime: DateTime, err: Throwable) extends StatusMessage
16 | case class JobErroredOut(jobId: String, endTime: DateTime, err: Throwable) extends StatusMessage
17 |
18 | /**
19 | * NOTE: For Subscribe, make sure to use `classOf[]` to get the Class for the case classes above.
20 | * Otherwise, `.getClass` will get the `java.lang.Class` of the companion object.
21 | */
22 | case class GetJobResult(jobId: String)
23 | case class JobResult(jobId: String, result: Any)
24 |
25 | case class Subscribe(jobId: String, receiver: ActorRef, events: Set[Class[_]]) {
26 | require(events.nonEmpty, "Must subscribe to at least one type of event!")
27 | }
28 | case class Unsubscribe(jobId: String, receiver: ActorRef) // all events for this jobId and receiving actor
29 |
30 | // errors
31 | case object NoSuchJobId
32 | case object NoSuchApplication
33 | case object NoSuchClass
34 | case object JobInitAlready
35 | case class NoJobSlotsAvailable(maxJobSlots: Int) // TODO: maybe rename this
36 | }
37 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/JarManager.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import ooyala.common.akka.InstrumentedActor
4 | import spark.jobserver.io.JobDAO
5 | import org.joda.time.DateTime
6 |
7 | // Messages to JarManager actor
8 | case class StoreJar(appName: String, jarBytes: Array[Byte])
9 | case object ListJars
10 |
11 | // Responses
12 | case object InvalidJar
13 | case object JarStored
14 |
15 | /**
16 | * An Actor that manages the jars stored by the job server. It's important that threads do not try to
17 | * load a class from a jar as a new one is replacing it, so using an actor to serialize requests is perfect.
18 | */
19 | class JarManager(jobDao: JobDAO) extends InstrumentedActor {
20 | override def wrappedReceive: Receive = {
21 | case ListJars => sender ! createJarsList()
22 |
23 | case StoreJar(appName, jarBytes) =>
24 | logger.info("Storing jar for app {}, {} bytes", appName, jarBytes.size)
25 | if (!JarUtils.validateJarBytes(jarBytes)) {
26 | sender ! InvalidJar
27 | } else {
28 | val uploadTime = DateTime.now()
29 | jobDao.saveJar(appName, uploadTime, jarBytes)
30 | sender ! JarStored
31 | }
32 | }
33 |
34 | private def createJarsList() = jobDao.getApps
35 | }
36 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/JarUtils.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import java.io.File
4 | import java.lang.ClassLoader
5 | import java.lang.reflect.Constructor
6 | import java.net.{ URL, URLClassLoader }
7 | import org.slf4j.LoggerFactory
8 |
9 | /**
10 | * A set of utilities for dynamically loading classes from a jar file, and saving the jar file.
11 | */
12 | object JarUtils {
13 | val logger = LoggerFactory.getLogger(getClass)
14 |
15 | /**
16 | * Loads a Scala object or class from a classloader.
17 | * See http://stackoverflow.com/questions/3216780/problem-reloading-a-jar-using-urlclassloader?lq=1
18 | * See http://stackoverflow.com/questions/8867766/scala-dynamic-object-class-loading
19 | *
20 | *
21 | * @param classOrObjectName must be the fully qualified name of the Scala object or class that
22 | * implements the SparkJob trait. If an object is used, do not include the
23 | * trailing '$'.
24 | * @param loader the ClassLoader to use to load the class or object. Typically a URLClassLoader.
25 | * @return Function0[C] to obtain the object/class. Calling the function will return a reference to
26 | * the object (for objects), or a new instance of a class (for classes) that implement the
27 | * SparkJob trait.
28 | */
29 | def loadClassOrObject[C](classOrObjectName: String, loader: ClassLoader): () => C = {
30 | def fallBackToClass(): () => C = {
31 | val constructor = loadConstructor[C](classOrObjectName, loader)
32 | () => constructor.newInstance()
33 | }
34 |
35 | // Try loading it as an object first, if that fails, then try it as a class
36 | try {
37 | val objectRef = loadObject[C](classOrObjectName + "$", loader)
38 | () => objectRef
39 | } catch {
40 | case e: java.lang.ClassNotFoundException => fallBackToClass()
41 | case e: java.lang.ClassCastException => fallBackToClass()
42 | case e: java.lang.NoSuchMethodException => fallBackToClass()
43 | case e: java.lang.NoSuchFieldException => fallBackToClass()
44 | }
45 | }
46 |
47 | private def loadConstructor[C](className: String, loader: ClassLoader): Constructor[C] = {
48 | logger.info("Loading class {} using loader {}", className: Any, loader)
49 | val loadedClass = loader.loadClass(className).asInstanceOf[Class[C]]
50 | val result = loadedClass.getConstructor()
51 | if (loadedClass.getClassLoader != loader) {
52 | logger.error("Wrong ClassLoader for class {}: Expected {} but got {}", loadedClass.getName,
53 | loader.toString, loadedClass.getClassLoader)
54 | }
55 | result
56 | }
57 |
58 | private def loadObject[C](objectName: String, loader: ClassLoader): C = {
59 | logger.info("Loading object {} using loader {}", objectName: Any, loader)
60 | val loadedClass = loader.loadClass(objectName)
61 | val objectRef = loadedClass.getField("MODULE$").get(null).asInstanceOf[C]
62 | if (objectRef.getClass.getClassLoader != loader) {
63 | logger.error("Wrong ClassLoader for object {}: Expected {} but got {}", objectRef.getClass.getName,
64 | loader.toString, objectRef.getClass.getClassLoader)
65 | }
66 | objectRef
67 | }
68 |
69 | def validateJarBytes(jarBytes: Array[Byte]): Boolean = {
70 | jarBytes.size > 4 &&
71 | // For now just check the first few bytes are the ZIP signature: 0x04034b50 little endian
72 | jarBytes(0) == 0x50 && jarBytes(1) == 0x4b && jarBytes(2) == 0x03 && jarBytes(3) == 0x04
73 | }
74 | }
75 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/JobCache.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import java.net.URL
4 | import ooyala.common.akka.metrics.MetricsWrapper
5 | import org.apache.spark.{SparkContext, SparkEnv}
6 | import org.joda.time.DateTime
7 | import spark.jobserver.io.JobDAO
8 | import spark.jobserver.util.{ContextURLClassLoader, LRUCache}
9 |
10 | case class JobJarInfo(constructor: () => SparkJob,
11 | className: String,
12 | jarFilePath: String)
13 |
14 | /**
15 | * A cache for SparkJob classes. A lot of times jobs are run repeatedly, and especially for low-latency
16 | * jobs, why retrieve the jar and load it every single time?
17 | */
18 | class JobCache(maxEntries: Int, dao: JobDAO, sparkContext: SparkContext, loader: ContextURLClassLoader) {
19 | private val cache = new LRUCache[(String, DateTime, String), JobJarInfo](getClass, maxEntries)
20 |
21 | /**
22 | * Retrieves the given SparkJob class from the cache if it's there, otherwise use the DAO to retrieve it.
23 | * @param appName the appName under which the jar was uploaded
24 | * @param uploadTime the upload time for the version of the jar wanted
25 | * @param classPath the fully qualified name of the class/object to load
26 | */
27 | def getSparkJob(appName: String, uploadTime: DateTime, classPath: String): JobJarInfo = {
28 |
29 | cache.get((appName, uploadTime, classPath), {
30 | val jarFilePath = new java.io.File(dao.retrieveJarFile(appName, uploadTime)).getAbsolutePath()
31 | sparkContext.addJar(jarFilePath) // Adds jar for remote executors
32 | loader.addURL(new URL("file:" + jarFilePath)) // Now jar added for local loader
33 | val constructor = JarUtils.loadClassOrObject[SparkJob](classPath, loader)
34 | JobJarInfo(constructor, classPath, jarFilePath)
35 | })
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/JobInfoActor.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.ActorRef
4 | import akka.pattern.ask
5 | import akka.util.Timeout
6 | import com.typesafe.config.Config
7 | import ooyala.common.akka.InstrumentedActor
8 | import scala.concurrent.Await
9 | import spark.jobserver.ContextSupervisor.{GetContext, GetAdHocContext}
10 | import spark.jobserver.io.JobDAO
11 |
12 | object JobInfoActor {
13 | // Requests
14 | case class GetJobStatuses(limit: Option[Int])
15 | case class GetJobStatus(jobId: String)
16 | case class GetJobConfig(jobId: String)
17 | case class StoreJobConfig(jobId: String, jobConfig: Config)
18 |
19 | // Responses
20 | case object JobConfigStored
21 | }
22 |
23 | class JobInfoActor(jobDao: JobDAO, contextSupervisor: ActorRef) extends InstrumentedActor {
24 | import CommonMessages._
25 | import JobInfoActor._
26 | import scala.concurrent.duration._
27 | import scala.util.control.Breaks._
28 | import context.dispatcher // for futures to work
29 |
30 | // Used in the asks (?) below to request info from contextSupervisor and resultActor
31 | implicit val ShortTimeout = Timeout(3 seconds)
32 |
33 | override def wrappedReceive: Receive = {
34 | case GetJobStatuses(limit) =>
35 | if (limit.isDefined) {
36 | sender ! jobDao.getJobInfosLimit(limit.get).values.toSeq.sortBy(_.startTime.toString())
37 | } else {
38 | sender ! jobDao.getJobInfos.values.toSeq.sortBy(_.startTime.toString())
39 | }
40 |
41 | case GetJobStatus(jobId) =>
42 | val jobInfoOpt = jobDao.getJobInfo(jobId)
43 | if (!jobInfoOpt.isDefined) {
44 | sender ! NoSuchJobId
45 | } else {
46 | sender ! jobInfoOpt.get
47 | }
48 |
49 | case GetJobResult(jobId) =>
50 | breakable {
51 | val jobInfoOpt = jobDao.getJobInfo(jobId)
52 | if (!jobInfoOpt.isDefined) {
53 | sender ! NoSuchJobId
54 | break
55 | }
56 |
57 | jobInfoOpt.filter { job => job.isRunning || job.isErroredOut }
58 | .foreach { jobInfo =>
59 | sender ! jobInfo
60 | break
61 | }
62 |
63 | // get the context from jobInfo
64 | val context = jobInfoOpt.get.contextName
65 |
66 | val future = (contextSupervisor ? ContextSupervisor.GetResultActor(context)).mapTo[ActorRef]
67 | val resultActor = Await.result(future, 3 seconds)
68 |
69 | val receiver = sender // must capture the sender since callbacks are run in a different thread
70 | for (result <- (resultActor ? GetJobResult(jobId))) {
71 | receiver ! result // a JobResult(jobId, result) object is sent
72 | }
73 | }
74 |
75 | case GetJobConfig(jobId) =>
76 | sender ! jobDao.getJobConfigs.get(jobId).getOrElse(NoSuchJobId)
77 |
78 | case StoreJobConfig(jobId, jobConfig) =>
79 | jobDao.saveJobConfig(jobId, jobConfig)
80 | sender ! JobConfigStored
81 | }
82 | }
83 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/JobResultActor.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.ActorRef
4 | import ooyala.common.akka.InstrumentedActor
5 | import ooyala.common.akka.metrics.MetricsWrapper
6 | import spark.jobserver.util.GuavaCacheUtils._
7 |
8 | import scala.collection.mutable
9 |
10 | /**
11 | * It is an actor to manage results that are returned from jobs.
12 | *
13 | * TODO: support multiple subscribers for same JobID
14 | */
15 | class JobResultActor extends InstrumentedActor {
16 | import CommonMessages._
17 |
18 | private val config = context.system.settings.config
19 |
20 | private val cache = buildCache[String, Any](config).withMetrics(getClass)
21 |
22 | private val subscribers = mutable.HashMap.empty[String, ActorRef] // subscribers
23 |
24 | // metrics
25 | private val metricSubscribers =
26 | MetricsWrapper.newGauge(getClass, "subscribers-size", subscribers.size)
27 |
28 | def wrappedReceive: Receive = {
29 | case Subscribe(jobId, receiver, events) =>
30 | if (events.contains(classOf[JobResult])) {
31 | subscribers(jobId) = receiver
32 | logger.info("Added receiver {} to subscriber list for JobID {}", receiver, jobId: Any)
33 | }
34 |
35 | case Unsubscribe(jobId, receiver) =>
36 | if (!subscribers.contains(jobId)) {
37 | sender ! NoSuchJobId
38 | } else {
39 | subscribers.remove(jobId)
40 | logger.info("Removed subscriber list for JobID {}", jobId)
41 | }
42 |
43 | case GetJobResult(jobId) =>
44 | sender ! Option(cache.getIfPresent(jobId)).map(JobResult(jobId, _)).getOrElse(NoSuchJobId)
45 |
46 | case JobResult(jobId, result) =>
47 | cache.put(jobId, result)
48 | logger.debug("Received job results for JobID {}", jobId)
49 | subscribers.get(jobId).foreach(_ ! JobResult(jobId, result))
50 | subscribers.remove(jobId)
51 | }
52 | }
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/JobServer.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.ActorSystem
4 | import akka.actor.Props
5 | import com.typesafe.config.{Config, ConfigFactory}
6 | import java.io.File
7 | import ooyala.common.akka.metrics.{DatadogConfigParser, MetricsLevel, MetricsWrapper}
8 | import org.slf4j.LoggerFactory
9 | import scala.util.Try
10 | import spark.jobserver.io.{JobDAO, JobDAOMetricsMonitor}
11 |
12 | /**
13 | * The Spark Job Server is a web service that allows users to submit and run Spark jobs, check status,
14 | * and view results.
15 | * It may offer other goodies in the future.
16 | * It only takes in one optional command line arg, a config file to override the default (and you can still
17 | * use -Dsetting=value to override)
18 | * -- Configuration --
19 | * {{{
20 | * spark {
21 | * master = "local"
22 | * jobserver {
23 | * port = 8090
24 | * }
25 | * }
26 | * }}}
27 | */
28 | object JobServer {
29 | val logger = LoggerFactory.getLogger(getClass)
30 |
31 | // Allow custom function to create ActorSystem. An example of why this is useful:
32 | // we can have something that stores the ActorSystem so it could be shut down easily later.
33 | def start(args: Array[String], makeSystem: Config => ActorSystem) {
34 | val defaultConfig = ConfigFactory.load()
35 | val config = if (args.length > 0) {
36 | val configFile = new File(args(0))
37 | if (!configFile.exists()) {
38 | println("Could not find configuration file " + configFile)
39 | sys.exit(1)
40 | }
41 | ConfigFactory.parseFile(configFile).withFallback(defaultConfig)
42 | } else {
43 | defaultConfig
44 | }
45 | logger.info("Starting JobServer with config {}", config.getConfig("spark").root.render())
46 | val port = config.getInt("spark.jobserver.port")
47 |
48 | // TODO: Hardcode for now to get going. Make it configurable later.
49 | val system = makeSystem(config)
50 | val clazz = Class.forName(config.getString("spark.jobserver.jobdao"))
51 | val ctor = clazz.getDeclaredConstructor(Class.forName("com.typesafe.config.Config"))
52 | val backingJobDao = ctor.newInstance(config).asInstanceOf[JobDAO]
53 | // Monitors JobDAO metrics if metrics level is appropriate.
54 | val metricsLevel = Try(MetricsLevel.valueOf(config.getInt("spark.jobserver.metrics.level")))
55 | .getOrElse(MetricsLevel.NONE)
56 | val jobDAO = metricsLevel match {
57 | case MetricsLevel.NONE => backingJobDao
58 | case level => JobDAOMetricsMonitor.newInstance(backingJobDao, level)
59 | }
60 |
61 | val jarManager = system.actorOf(Props(classOf[JarManager], jobDAO), "jar-manager")
62 | val supervisor = system.actorOf(Props(classOf[LocalContextSupervisorActor], jobDAO), "context-supervisor")
63 | val jobInfo = system.actorOf(Props(classOf[JobInfoActor], jobDAO, supervisor), "job-info")
64 | val sparkWebUi = system.actorOf(Props(classOf[SparkWebUiActor]), "spark-web-ui")
65 | // Create initial contexts
66 | supervisor ! ContextSupervisor.AddContextsFromConfig
67 | new WebApi(system, config, port, jarManager, supervisor, jobInfo, sparkWebUi).start()
68 |
69 | // Setups and starts datadog reporting
70 | val datadogConfig = DatadogConfigParser.parse(config)
71 | MetricsWrapper.startDatadogReporter(datadogConfig)
72 | }
73 |
74 | def main(args: Array[String]) {
75 | start(args, config => ActorSystem("JobServer", config))
76 | }
77 | }
78 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/JobServerNamedRdds.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.ActorRef
4 | import akka.util.Timeout
5 | import org.apache.spark.rdd.RDD
6 | import org.apache.spark.storage.StorageLevel
7 | import scala.concurrent.Await
8 |
9 | /**
10 | * An implementation of [[NamedRddSupport]] API for the Job Server.
11 | * Note that this contains code that executes on the same thread as the job.
12 | * Another part of this system is the rddManager, which is an actor which manages the concurrent
13 | * update of shared RDD state.
14 | */
15 | class JobServerNamedRdds(val rddManager: ActorRef) extends NamedRdds {
16 | import RddManagerActorMessages._
17 |
18 | require(rddManager != null, "rddManager ActorRef must not be null!")
19 |
20 | def getOrElseCreate[T](name: String, rddGen: => RDD[T])
21 | (implicit timeout: Timeout = defaultTimeout): RDD[T] = {
22 | import akka.pattern.ask
23 |
24 | val future = rddManager.ask(GetOrElseCreateRddRequest(name))(timeout)
25 | val result: RDD[T] = Await.result(future, timeout.duration) match {
26 | case Left(error: Throwable) =>
27 | throw new RuntimeException("Failed to get named RDD '" + name + "'", error)
28 | case Right(rdd: RDD[T]) => refreshRdd(rdd)
29 | case None =>
30 | // Try to generate the RDD and send the result of the operation to the rddManager.
31 | try {
32 | val rdd = createRdd(rddGen, name)
33 | rddManager ! CreateRddResult(name, Right(rdd))
34 | rdd
35 | } catch {
36 | case error: Throwable =>
37 | rddManager ! CreateRddResult(name, Left(error))
38 | throw new RuntimeException("Failed to create named RDD '" + name + "'", error)
39 | }
40 | }
41 | result
42 | }
43 |
44 | def get[T](name: String)(implicit timeout: Timeout = defaultTimeout): Option[RDD[T]] = {
45 | import akka.pattern.ask
46 |
47 | val future = rddManager ? GetRddRequest(name)
48 | Await.result(future, timeout.duration) match {
49 | case rddOpt: Option[RDD[T]] @unchecked => rddOpt.map { rdd => refreshRdd(rdd) }
50 | }
51 | }
52 |
53 | def update[T](name: String, rddGen: => RDD[T]): RDD[T] = {
54 | val rdd = createRdd(rddGen, name)
55 | rddManager ! CreateRddResult(name, Right(rdd))
56 | rdd
57 | }
58 |
59 | def destroy(name: String) {
60 | rddManager ! DestroyRdd(name)
61 | }
62 |
63 | def getNames()(implicit timeout: Timeout = defaultTimeout): Iterable[String] = {
64 | import akka.pattern.ask
65 |
66 | val future = rddManager ? GetRddNames
67 | Await.result(future, timeout.duration) match {
68 | case answer: Iterable[String] @unchecked => answer
69 | }
70 | }
71 |
72 | /**
73 | * Creates an RDD by calling the given generator, sets its name, persists it with the given storage level,
74 | * and optionally forces its contents to be computed.
75 | * @param rddGen a 0-ary function which will be called to generate the RDD in the caller's thread.
76 | * @param name the name to assign to the RDD.
77 | * @param storageLevel the storage level to persist the RDD with. Default: StorageLevel.MEMORY_ONLY.
78 | * @param forceComputation if true, forces the RDD to be computed by calling count().
79 | * @throws java.lang.IllegalArgumentException if forceComputation == true &&
80 | * storageLevel == StorageLevel.NONE
81 | */
82 | private def createRdd[T](rddGen: => RDD[T],
83 | name: String,
84 | forceComputation: Boolean = true,
85 | storageLevel: StorageLevel = defaultStorageLevel): RDD[T] = {
86 | require(!forceComputation || storageLevel != StorageLevel.NONE,
87 | "forceComputation implies storageLevel != NONE")
88 | val rdd = rddGen
89 | rdd.setName(name)
90 | if (storageLevel != StorageLevel.NONE) rdd.persist(storageLevel)
91 | // TODO: figure out if there is a better way to force the RDD to be computed
92 | if (forceComputation) rdd.count()
93 | rdd
94 | }
95 |
96 | /** Calls rdd.persist(), which updates the RDD's cached timestamp, meaning it won't get
97 | * garbage collected by Spark for some time.
98 | * @param rdd the RDD
99 | */
100 | private def refreshRdd[T](rdd: RDD[T]): RDD[T] = rdd.persist(rdd.getStorageLevel)
101 | }
102 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/JobStatusActor.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.ActorRef
4 | import com.codahale.metrics.Meter
5 | import ooyala.common.akka.metrics.MetricsWrapper
6 | import ooyala.common.akka.InstrumentedActor
7 | import scala.collection.mutable
8 | import scala.util.Try
9 | import spark.jobserver.io.{ JobInfo, JobDAO }
10 |
11 | object JobStatusActor {
12 | case class JobInit(jobInfo: JobInfo)
13 | case class GetRunningJobStatus()
14 | }
15 |
16 | /**
17 | * It is an actor to manage job status updates
18 | *
19 | */
20 | class JobStatusActor(jobDao: JobDAO) extends InstrumentedActor {
21 | import CommonMessages._
22 | import JobStatusActor._
23 | import spark.jobserver.util.DateUtils.dateTimeToScalaWrapper
24 |
25 | // jobId to its JobInfo
26 | private val infos = new mutable.HashMap[String, JobInfo]
27 | // subscribers
28 | private val subscribers = new mutable.HashMap[String, mutable.MultiMap[Class[_], ActorRef]]
29 |
30 | // metrics
31 | val metricNumSubscriptions = MetricsWrapper.newGauge(getClass, "num-subscriptions", subscribers.size)
32 | val metricNumJobInfos = MetricsWrapper.newGauge(getClass, "num-running-jobs", infos.size)
33 | val metricJobSubmissionRate = MetricsWrapper.newMeter(getClass, "job-submission-rate")
34 | val metricStatusRates = mutable.HashMap.empty[String, Meter]
35 |
36 | // timer for job latency
37 | private val jobLatencyTimer = MetricsWrapper.newTimer(getClass, "job-latency");
38 | // timer context to measure the job latency (jobId to Timer.Context mapping)
39 | private val latencyTimerContextMap = new mutable.HashMap[String, com.codahale.metrics.Timer.Context]
40 |
41 | override def wrappedReceive: Receive = {
42 | case GetRunningJobStatus =>
43 | sender ! infos.values.toSeq.sortBy(_.startTime) // TODO(kelvinchu): Use toVector instead in Scala 2.10
44 |
45 | case Unsubscribe(jobId, receiver) =>
46 | subscribers.get(jobId) match {
47 | case Some(jobSubscribers) =>
48 | jobSubscribers.transform { case (event, receivers) => receivers -= receiver }
49 | .retain { case (event, receivers) => receivers.nonEmpty }
50 | if (jobSubscribers.isEmpty) subscribers.remove(jobId)
51 | case None =>
52 | // TODO: The message below is named poorly. There may be such a job id, there are just no
53 | // registered subscribers for this job id.
54 | logger.error("No such job id " + jobId)
55 | sender ! NoSuchJobId
56 | }
57 |
58 | case Subscribe(jobId, receiver, events) =>
59 | // Subscription is independent of job life cycles. So, don't need to check infos.
60 | val jobSubscribers = subscribers.getOrElseUpdate(jobId, newMultiMap())
61 | events.foreach { event => jobSubscribers.addBinding(event, receiver) }
62 |
63 | case JobInit(jobInfo) =>
64 | // TODO (kelvinchu): Check if the jobId exists in the persistence store already
65 | if (!infos.contains(jobInfo.jobId)) {
66 | infos(jobInfo.jobId) = jobInfo
67 | metricJobSubmissionRate.mark()
68 | } else {
69 | sender ! JobInitAlready
70 | }
71 |
72 | case msg: JobStarted =>
73 | latencyTimerContextMap(msg.jobId) = jobLatencyTimer.time();
74 | processStatus(msg, "started") {
75 | case (info, msg) =>
76 | info.copy(startTime = msg.startTime)
77 | }
78 |
79 | case msg: JobFinished =>
80 | stopTimer(msg.jobId)
81 | processStatus(msg, "finished OK", remove = true) {
82 | case (info, msg) =>
83 | info.copy(endTime = Some(msg.endTime))
84 | }
85 |
86 | case msg: JobValidationFailed =>
87 | processStatus(msg, "validation failed", remove = true) {
88 | case (info, msg) =>
89 | info.copy(endTime = Some(msg.endTime), error = Some(msg.err))
90 | }
91 |
92 | case msg: JobErroredOut =>
93 | stopTimer(msg.jobId)
94 | processStatus(msg, "finished with an error", remove = true) {
95 | case (info, msg) =>
96 | info.copy(endTime = Some(msg.endTime), error = Some(msg.err))
97 | }
98 | }
99 |
100 | private def stopTimer(jobId: String) {
101 | latencyTimerContextMap.get(jobId).foreach { timerContext =>
102 | timerContext.stop()
103 | latencyTimerContextMap.remove(jobId)
104 | }
105 | }
106 |
107 | private def processStatus[M <: StatusMessage](msg: M, logMessage: String, remove: Boolean = false)
108 | (infoModifier: (JobInfo, M) => JobInfo) {
109 | if (infos.contains(msg.jobId)) {
110 | infos(msg.jobId) = infoModifier(infos(msg.jobId), msg)
111 | logger.info("Job {} {}", msg.jobId: Any, logMessage)
112 | jobDao.saveJobInfo(infos(msg.jobId))
113 | publishMessage(msg.jobId, msg)
114 | updateMessageRate(msg)
115 | if (remove) infos.remove(msg.jobId)
116 | } else {
117 | logger.error("No such job id " + msg.jobId)
118 | sender ! NoSuchJobId
119 | }
120 | }
121 |
122 | private def updateMessageRate(msg: StatusMessage) {
123 | val msgClass = msg.getClass.getCanonicalName
124 |
125 | lazy val getShortName = Try(msgClass.split('.').last).toOption.getOrElse(msgClass)
126 |
127 | metricStatusRates.getOrElseUpdate(msgClass, MetricsWrapper.newMeter(getClass, getShortName + ".messages")).mark()
128 | }
129 |
130 | private def publishMessage(jobId: String, message: StatusMessage) {
131 | for (
132 | jobSubscribers <- subscribers.get(jobId);
133 | receivers <- jobSubscribers.get(message.getClass);
134 | receiver <- receivers
135 | ) {
136 | receiver ! message
137 | }
138 | }
139 |
140 | private def newMultiMap(): mutable.MultiMap[Class[_], ActorRef] =
141 | new mutable.HashMap[Class[_], mutable.Set[ActorRef]] with mutable.MultiMap[Class[_], ActorRef]
142 | }
143 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/RddManagerActor.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.ActorRef
4 | import ooyala.common.akka.InstrumentedActor
5 | import ooyala.common.akka.metrics.MetricsWrapper
6 | import org.apache.spark.rdd.RDD
7 | import org.apache.spark.SparkContext
8 | import scala.collection.mutable
9 |
10 | object RddManagerActorMessages {
11 | // Message which asks to retrieve an RDD by name. If no such RDD is found, None will be returned.
12 | case class GetRddRequest(name: String)
13 |
14 | // Message which asks to retrieve an RDD by name. Different from GetRddRequest, because it tells the
15 | // RddManager that the client is willing to create the RDD with this name if one does not already exist.
16 | case class GetOrElseCreateRddRequest(name: String)
17 |
18 | // Message which tells the RddManager that a new RDD has been created, or that RDD generation failed.
19 | case class CreateRddResult(name: String, rddOrError: Either[Throwable, RDD[_]])
20 |
21 | // Message which tells the RddManager that an RDD should be destroyed and all of its cached blocks removed
22 | case class DestroyRdd(name: String)
23 |
24 | // Message which asks for the names of all RDDs currently managed by the RddManager
25 | case object GetRddNames
26 | }
27 |
28 | class RddManagerActor(sparkContext: SparkContext) extends InstrumentedActor {
29 | import RddManagerActorMessages._
30 |
31 | private val namesToIds = new mutable.HashMap[String, Int]()
32 | private val waiters =
33 | new mutable.HashMap[String, mutable.Set[ActorRef]] with mutable.MultiMap[String, ActorRef]
34 | private val inProgress = mutable.Set[String]()
35 | // Metrics
36 | private val rddNumGauge = MetricsWrapper.newGauge(getClass, "num-rdds", namesToIds.size)
37 |
38 | def wrappedReceive: Receive = {
39 | case GetRddRequest(name) => sender ! getExistingRdd(name)
40 |
41 | case GetOrElseCreateRddRequest(name) if inProgress.contains(name) =>
42 | logger.info("RDD [{}] already being created, actor {} added to waiters list", name: Any, sender.path)
43 | waiters.addBinding(name, sender)
44 |
45 | case GetOrElseCreateRddRequest(name) => getExistingRdd(name) match {
46 | case Some(rdd) => sender ! Right(rdd)
47 | case None =>
48 | logger.info("RDD [{}] not found, starting creation", name)
49 | inProgress.add(name)
50 | sender ! None
51 | }
52 |
53 | // TODO: log the error?
54 | case CreateRddResult(name, Left(error)) => notifyAndClearWaiters(name, Left(error))
55 |
56 | case CreateRddResult(name, Right(rdd)) =>
57 | val oldRddOption = getExistingRdd(name)
58 | namesToIds(name) = rdd.id
59 | notifyAndClearWaiters(name, Right(rdd))
60 | // Note: unpersist the old rdd we just replaced, if there was one
61 | if (oldRddOption.isDefined && oldRddOption.get.id != rdd.id) {
62 | oldRddOption.get.unpersist(blocking = false)
63 | }
64 |
65 | case DestroyRdd(name) => getExistingRdd(name).foreach { rdd =>
66 | namesToIds.remove(name)
67 | rdd.unpersist(blocking = false)
68 | }
69 |
70 | case GetRddNames =>
71 | val persistentRdds = sparkContext.getPersistentRDDs
72 | val result = namesToIds.collect { case (name, id) if persistentRdds.contains(id) => name }
73 | // optimization: can remove stale names from our map if the SparkContext has unpersisted them.
74 | (namesToIds.keySet -- result).foreach { staleName => namesToIds.remove(staleName) }
75 | sender ! result
76 | }
77 |
78 | private def getExistingRdd(name: String): Option[RDD[_]] =
79 | namesToIds.get(name).flatMap { id => sparkContext.getPersistentRDDs.get(id) } match {
80 | case Some(rdd) => Some(rdd)
81 | case None =>
82 | // If this happens, maybe we never knew about this RDD, or maybe we had a name -> id mapping, but
83 | // spark's MetadataCleaner has evicted this RDD from the cache because it was too old, and we need
84 | // to forget about it. Remove it from our names -> ids map and respond as if we never knew about it.
85 | namesToIds.remove(name)
86 | None
87 | }
88 |
89 | private def notifyAndClearWaiters(name: String, message: Any) {
90 | waiters.get(name).foreach { actors => actors.foreach { actor => actor ! message } }
91 | waiters.remove(name) // Note: this removes all bindings for the key in the MultiMap
92 | inProgress.remove(name) // this RDD is no longer being computed, clear in progress flag
93 | }
94 | }
95 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/SparkWebUiActor.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.ActorSystem
4 | import akka.io.IO
5 | import akka.pattern.ask
6 | import akka.util.Timeout
7 | import ooyala.common.akka.InstrumentedActor
8 | import scala.util.{Success, Failure}
9 | import scala.concurrent.Future
10 | import spark.jobserver.SparkWebUiActor.{SparkWorkersErrorInfo, SparkWorkersInfo, GetWorkerStatus}
11 | import spray.can.Http
12 | import spray.client.pipelining.{Get, sendReceive, SendReceive}
13 | import spray.http.{HttpResponse, HttpRequest}
14 |
15 | object SparkWebUiActor {
16 | // Requests
17 | case class GetWorkerStatus()
18 |
19 | // Responses
20 | case class SparkWorkersInfo(alive: Int, dead: Int)
21 | case class SparkWorkersErrorInfo(message :String)
22 | }
23 | /**
24 | * This actor pulls Spark worker status info (ALIVE, DEAD etc) from Spark admin web ui
25 | * Collecting worker info from HTML page is not ideal.
26 | * But at this time Spark does not provide public API yet to expose worker status.
27 | * Also, the current implementation only works for Spark standalone mode
28 | */
29 | class SparkWebUiActor extends InstrumentedActor {
30 | import actorSystem.dispatcher // execution context for futures
31 | import scala.concurrent.duration._
32 |
33 | implicit val actorSystem: ActorSystem = context.system
34 |
35 | val config = context.system.settings.config
36 |
37 | val sparkWebHostUrls: Array[String] = getSparkHostName()
38 | val sparkWebHostPort = config.getInt("spark.webUrlPort")
39 |
40 | // implicit timeout value for ? of IO(Http)
41 | implicit val shortTimeout = Timeout(3 seconds)
42 | // get a pipeline every time we need it since pipeline could time out or die.
43 | // The connector will be re-used if it exists so the cost is low:
44 | // from http://spray.io/documentation/1.1-M8/spray-can/http-client/host-level/
45 | // If there is no connector actor running for the given combination of hostname,
46 | // port and settings spray-can will start a new one,
47 | // otherwise the existing one is going to be re-used.
48 | def pipelines: Array[Future[SendReceive]] = sparkWebHostUrls.map(url =>
49 | for (
50 | Http.HostConnectorInfo(connector, _) <- IO(Http) ? Http.HostConnectorSetup(url, port = sparkWebHostPort)
51 | ) yield sendReceive(connector)
52 | )
53 |
54 | override def postStop() {
55 | logger.info("Shutting down actor system for SparkWebUiActor")
56 | }
57 |
58 | override def wrappedReceive: Receive = {
59 | case GetWorkerStatus() =>
60 | val request = Get("/")
61 |
62 | val theSender = sender
63 |
64 | pipelines.map {pipeline =>
65 | val responseFuture: Future[HttpResponse] = pipeline.flatMap(_(request))
66 | responseFuture onComplete {
67 | case Success(httpResponse) =>
68 | val content = httpResponse.entity.asString.replace('\n', ' ');
69 |
70 | // regex to detect the #running tasks
71 | val runningTaskRegex = """.*
Applications:\s*(\d+)\s+Running.*""".r
72 | content match {
73 | case runningTaskRegex(runningTaskStr) =>
74 | if (runningTaskStr != null && runningTaskStr.length > 0 && runningTaskStr.toInt > 0) {
75 | // we believe it is a active master if it has active running tasks
76 | // we only check the workers on active master
77 | val aliveWorkerNum = "
ALIVE
".r.findAllIn(content).length
78 | val deadWorkerNum = "
DEAD
".r.findAllIn(content).length
79 |
80 | theSender ! SparkWorkersInfo(aliveWorkerNum, deadWorkerNum)
81 | }
82 | case _ => throw new RuntimeException("Could not parse HTML response: '" + content + "'")
83 | }
84 | case Failure(error) =>
85 | val msg = "Failed to retrieve Spark web UI: " + error.getMessage
86 | logger.error(msg)
87 | }
88 | }
89 | }
90 |
91 | def getSparkHostName(): Array[String] = {
92 | val master = config.getString("spark.master")
93 | // Regular expression used for local[N] and local[*] master formats
94 | val LOCAL_N_REGEX = """local\[([0-9\*]+)\]""".r
95 | // Regular expression for connecting to Spark deploy clusters
96 | val SPARK_REGEX = """spark://(.*)""".r
97 |
98 | master match {
99 | case "localhost" | "local" | LOCAL_N_REGEX(_) => Array("localhost")
100 | case SPARK_REGEX(sparkUrl) =>
101 | val masterUrls = sparkUrl.split(",").map(s => {
102 | val splits = s.split(":")
103 | splits(0)
104 | })
105 | masterUrls
106 | case _ => throw new RuntimeException("Could not parse Master URL: '" + master + "'")
107 | }
108 | }
109 |
110 |
111 | }
112 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/io/JobCacheDAO.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.io
2 |
3 | import java.io._
4 |
5 | import com.typesafe.config._
6 | import org.joda.time.DateTime
7 | import org.slf4j.LoggerFactory
8 | import spark.jobserver.util.LRUCache
9 |
10 | import scala.collection.mutable
11 | import scala.util.Try
12 |
13 | /**
14 | * Store job information (JobInfo, config) in memory cache only, but jar files still in file system.
15 | *
16 | *
17 | * @param config
18 | */
19 | class JobCacheDAO(config: Config) extends JobDAO {
20 | private val logger = LoggerFactory.getLogger(getClass)
21 |
22 | private val charsetName = "UTF-8"
23 |
24 | // appName to its set of upload times. Decreasing times in the seq.
25 | private val apps = mutable.HashMap.empty[String, Seq[DateTime]]
26 | // jobId to its (JobInfo, config)
27 | private val jobs = new LRUCache[String, (JobInfo, Config)](this.getClass,
28 | Try(config.getInt("spark.jobserver.cachedao.size")).getOrElse(500))
29 |
30 | private val rootDir = getOrElse(config.getString("spark.jobserver.cachedao.rootdir"),
31 | "/tmp/spark-jobserver/cachedao/data")
32 | private val rootDirFile = new File(rootDir)
33 | logger.info("rootDir is " + rootDirFile.getAbsolutePath)
34 |
35 | private val jarsFile = new File(rootDirFile, "jars.data")
36 | private var jarsOutputStream: DataOutputStream = null
37 |
38 | init()
39 |
40 | private def init() {
41 | // create the date directory if it doesn't exist
42 | if (!rootDirFile.exists()) {
43 | if (!rootDirFile.mkdirs()) {
44 | throw new RuntimeException("Could not create directory " + rootDir)
45 | }
46 | }
47 |
48 | // read back all apps info during startup
49 | if (jarsFile.exists()) {
50 | val in = new DataInputStream(new BufferedInputStream(new FileInputStream(jarsFile)))
51 | try {
52 | while (true) {
53 | val jarInfo = readJarInfo(in)
54 | addJar(jarInfo.appName, jarInfo.uploadTime)
55 | }
56 | } catch {
57 | case e: EOFException => // do nothing
58 |
59 | } finally {
60 | in.close()
61 | }
62 | }
63 |
64 | // Don't buffer the stream. I want the apps meta data log directly into the file.
65 | // Otherwise, server crash will lose the buffer data.
66 | jarsOutputStream = new DataOutputStream(new FileOutputStream(jarsFile, true))
67 | }
68 |
69 | override def saveJar(appName: String, uploadTime: DateTime, jarBytes: Array[Byte]) {
70 | // The order is important. Save the jar file first and then log it into jobsFile.
71 | val outFile = new File(rootDir, createJarName(appName, uploadTime) + ".jar")
72 | val bos = new BufferedOutputStream(new FileOutputStream(outFile))
73 | try {
74 | logger.debug("Writing {} bytes to file {}", jarBytes.size, outFile.getPath)
75 | bos.write(jarBytes)
76 | bos.flush()
77 | } finally {
78 | bos.close()
79 | }
80 |
81 | // log it into jobsFile
82 | writeJarInfo(jarsOutputStream, JarInfo(appName, uploadTime))
83 |
84 | // track the new jar in memory
85 | addJar(appName, uploadTime)
86 | }
87 |
88 | private def writeJarInfo(out: DataOutputStream, jarInfo: JarInfo) {
89 | out.writeUTF(jarInfo.appName)
90 | out.writeLong(jarInfo.uploadTime.getMillis)
91 | }
92 |
93 | private def readJarInfo(in: DataInputStream) = JarInfo(in.readUTF, new DateTime(in.readLong))
94 |
95 | private def addJar(appName: String, uploadTime: DateTime) {
96 | if (apps.contains(appName)) {
97 | apps(appName) = uploadTime +: apps(appName) // latest time comes first
98 | } else {
99 | apps(appName) = Seq(uploadTime)
100 | }
101 | }
102 |
103 | def getApps: Map[String, DateTime] = apps.map {
104 | case (appName, uploadTimes) =>
105 | appName -> uploadTimes.head
106 | }.toMap
107 |
108 | override def retrieveJarFile(appName: String, uploadTime: DateTime): String =
109 | new File(rootDir, createJarName(appName, uploadTime) + ".jar").getAbsolutePath
110 |
111 | private def createJarName(appName: String, uploadTime: DateTime): String =
112 | appName + "-" + uploadTime.toString().replace(':', '_')
113 |
114 | override def saveJobInfo(jobInfo: JobInfo) {
115 | jobs.synchronized {
116 | jobs.get(jobInfo.jobId) match {
117 | case Some((_, config)) => jobs.put(jobInfo.jobId, (jobInfo, config))
118 | case None => jobs.put(jobInfo.jobId, (jobInfo, null))
119 | }
120 | }
121 | }
122 |
123 | private def readError(in: DataInputStream) = {
124 | val error = in.readUTF()
125 | if (error == "") None else Some(new Throwable(error))
126 | }
127 |
128 | override def getJobInfos: Map[String, JobInfo] = jobs.toMap.map {case (id, (jobInfo, _)) => (id, jobInfo)}
129 |
130 | override def getJobInfosLimit(limit: Int): Map[String, JobInfo] = getJobInfos.takeRight(limit)
131 |
132 | override def getJobInfo(jobId: String): Option[JobInfo] = jobs.get(jobId).map(_._1)
133 |
134 | override def saveJobConfig(jobId: String, config: Config) {
135 | jobs.synchronized {
136 | jobs.get(jobId) match {
137 | case Some((jobInfo, _)) => jobs.put(jobId, (jobInfo, config))
138 | case None => jobs.put(jobId, (null, config))
139 | }
140 | }
141 | }
142 |
143 | override def getJobConfigs: Map[String, Config] = jobs.toMap.map {case (id, (_, config)) => (id, config)}
144 |
145 | }
146 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/io/JobDAO.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.io
2 |
3 | import com.typesafe.config._
4 | import org.joda.time.{ Duration, DateTime }
5 |
6 | // Uniquely identifies the jar used to run a job
7 | case class JarInfo(appName: String, uploadTime: DateTime)
8 |
9 | // Both a response and used to track job progress
10 | // NOTE: if endTime is not None, then the job has finished.
11 | case class JobInfo(jobId: String, contextName: String,
12 | jarInfo: JarInfo, classPath: String,
13 | startTime: DateTime, endTime: Option[DateTime],
14 | error: Option[Throwable]) {
15 | def jobLengthMillis: Option[Long] = endTime.map { end => new Duration(startTime, end).getMillis() }
16 |
17 | def isRunning: Boolean = !endTime.isDefined
18 | def isErroredOut: Boolean = endTime.isDefined && error.isDefined
19 | }
20 |
21 | /**
22 | * Core trait for data access objects for persisting data such as jars, applications, jobs, etc.
23 | */
24 | trait JobDAO {
25 | /**
26 | * Persist a jar.
27 | *
28 | * @param appName
29 | * @param uploadTime
30 | * @param jarBytes
31 | */
32 | def saveJar(appName: String, uploadTime: DateTime, jarBytes: Array[Byte])
33 |
34 | /**
35 | * Return all applications name and their last upload times.
36 | *
37 | * @return
38 | */
39 | def getApps: Map[String, DateTime]
40 |
41 | /**
42 | * TODO(kelvinchu): Remove this method later when JarManager doesn't use it anymore.
43 | *
44 | * @param appName
45 | * @param uploadTime
46 | * @return the local file path of the retrieved jar file.
47 | */
48 | def retrieveJarFile(appName: String, uploadTime: DateTime): String
49 |
50 | /**
51 | * Persist a job info.
52 | *
53 | * @param jobInfo
54 | */
55 | def saveJobInfo(jobInfo: JobInfo)
56 |
57 | /**
58 | * Return all job ids to their job info.
59 | *
60 | * @return
61 | */
62 | def getJobInfos: Map[String, JobInfo]
63 |
64 | /**
65 | * Get jobInfo for the first limit job
66 | *
67 | * @param limit the number of jobInfo returned
68 | * @return Map[String, JobInfo]
69 | */
70 | def getJobInfosLimit(limit: Int): Map[String, JobInfo]
71 |
72 | /**
73 | * Get jobInfo for a given job id
74 | *
75 | * @param jobId
76 | * @return Option[JobInfo]
77 | */
78 | def getJobInfo(jobId: String): Option[JobInfo]
79 |
80 | /**
81 | * Persist a job configuration along with provided jobId.
82 | *
83 | * @param jobId
84 | * @param jobConfig
85 | */
86 | def saveJobConfig(jobId: String, jobConfig: Config)
87 |
88 | /**
89 | * Return all job ids to their job configuration.
90 | *
91 | * @return
92 | */
93 | def getJobConfigs: Map[String, Config]
94 |
95 | /**
96 | * Returns the last upload time for a given app name.
97 | * @return Some(lastUploadedTime) if the app exists and the list of times is nonempty, None otherwise
98 | */
99 | def getLastUploadTime(appName: String): Option[DateTime] =
100 | getApps.get(appName)
101 |
102 | /**
103 | * Return all contexts
104 | */
105 | def getContexts: Map[String, Config] = {
106 | Map()
107 | }
108 |
109 | /**
110 | * Return the context configuration
111 | *
112 | * @param name the context name
113 | */
114 | def getContextConfig(name: String): Option[Config] = {
115 | None
116 | }
117 |
118 | /**
119 | * Persist a context configuration
120 | *
121 | * @param name the context name
122 | * @param config the context configuration
123 | */
124 | def saveContextConfig(name: String, config: Config) = {
125 | }
126 |
127 | /**
128 | * A safe API for getting values from Typesafe Config, will return a default if the
129 | * value is missing. If the value is badly formatted, error still goes through.
130 | */
131 | def getOrElse[T](getter: => T, default: T): T = {
132 | try getter catch {
133 | case e: ConfigException.Missing => default
134 | }
135 | }
136 | }
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/io/JobDAOMetricsMonitor.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.io
2 |
3 | import com.codahale.metrics.Meter
4 | import com.typesafe.config.{Config, ConfigRenderOptions}
5 | import java.lang.reflect.{InvocationHandler, Method, Proxy}
6 | import ooyala.common.akka.metrics.MetricsLevel
7 | import ooyala.common.akka.metrics.MetricsWrapper.{newHistogram, newMeter}
8 |
9 | /**
10 | * This invocation handler monitors the job DAO and updates the metrics.
11 | *
12 | * @param jobDao the actual job DAO to be monitored
13 | * @param metricsLevel the level of details of metrics to monitor
14 | */
15 | class JobDAOMetricsMonitor(jobDao: JobDAO, metricsLevel: MetricsLevel) extends InvocationHandler {
16 | // Basic metrics
17 | private val basicMetrics: Map[String, Meter] = {
18 | val metrics = collection.mutable.Map[String, Meter]()
19 |
20 | classOf[JobDAO].getMethods.foreach {
21 | method => {
22 | metrics(method.getName) = newMeter(getClass, method.getName)
23 | }
24 | }
25 |
26 | metrics.toMap
27 | }
28 | // Fine metrics
29 | private lazy val metricJarInfoSize = newHistogram(getClass, "JarSize")
30 | private lazy val metricJobInfoSize = newHistogram(getClass, "JobInfoSize")
31 | private lazy val metricJobConfigSize = newHistogram(getClass, "JobConfigSize")
32 | private lazy val metricContextConfigSize = newHistogram(getClass, "ContextConfigSize")
33 | // Handlers for updating fine metrics
34 | private val fineMetricsHandlers = Map[String, Array[AnyRef] => Unit](
35 | "saveJar" -> (
36 | (args: Array[AnyRef]) => {
37 | // Updates metric for jar size
38 | val appName = args(0).asInstanceOf[String]
39 | val jarBytes = args(2).asInstanceOf[Array[Byte]]
40 | metricJarInfoSize.update(appName.length + jarBytes.size)
41 | }),
42 |
43 | "saveJobConfig" -> (
44 | (args: Array[AnyRef]) => {
45 | // Updates metric for job config size
46 | val jobConfig = args(1).asInstanceOf[Config]
47 | val configStr = jobConfig.root().render(ConfigRenderOptions.concise())
48 | metricJobConfigSize.update(configStr.length)
49 | }),
50 |
51 | "saveJobInfo" -> (
52 | (args: Array[AnyRef]) => {
53 | // Updates metric for job info size
54 | val jobInfo = args(0).asInstanceOf[JobInfo]
55 | val jobInfoSize = jobInfo.contextName.length + jobInfo.classPath.length +
56 | jobInfo.error.map(_.getMessage.length).getOrElse(0)
57 | metricJobInfoSize.update(jobInfoSize)
58 | }),
59 |
60 | "saveContextConfig" -> (
61 | (args: Array[AnyRef]) => {
62 | // Updates metric for context config size
63 | val name = args(0).asInstanceOf[String]
64 | val config = args(1).asInstanceOf[Config]
65 | val configStr = config.root().render(ConfigRenderOptions.concise())
66 | metricContextConfigSize.update(name.length + configStr.length)
67 | }))
68 |
69 | override def invoke(proxy: AnyRef, method: Method, args: Array[AnyRef]): AnyRef = {
70 | val methodName = method.getName
71 |
72 | // Monitors basic metrics
73 | basicMetrics.get(methodName).foreach {
74 | _.mark
75 | }
76 |
77 | // Monitors more detailed metrics
78 | if (metricsLevel >= MetricsLevel.FINE) {
79 | fineMetricsHandlers.get(methodName).foreach {
80 | _(args)
81 | }
82 | }
83 |
84 | method.invoke(jobDao, args: _*)
85 | }
86 | }
87 |
88 | object JobDAOMetricsMonitor {
89 |
90 | def newInstance(jobDao: JobDAO, metricsLevel: MetricsLevel): JobDAO = {
91 | val metricsMonitor = new JobDAOMetricsMonitor(jobDao, metricsLevel)
92 |
93 | return Proxy.newProxyInstance(getClass.getClassLoader, Array(classOf[JobDAO]), metricsMonitor)
94 | .asInstanceOf[JobDAO]
95 | }
96 | }
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/util/ContextURLClassLoader.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.util
2 |
3 | import java.net.{URLClassLoader, URL}
4 | import org.slf4j.LoggerFactory
5 |
6 | /**
7 | * The addURL method in URLClassLoader is protected. We subclass it to make this accessible.
8 | * NOTE: This is copied from Spark's ExecutorURLClassLoader, which is private[spark].
9 | */
10 | class ContextURLClassLoader(urls: Array[URL], parent: ClassLoader)
11 | extends URLClassLoader(urls, parent) {
12 |
13 | val logger = LoggerFactory.getLogger(getClass)
14 |
15 | override def addURL(url: URL) {
16 | if (!getURLs.contains(url)) {
17 | super.addURL(url)
18 | logger.info("Added URL " + url + " to ContextURLClassLoader")
19 | }
20 | }
21 | }
22 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/util/DateUtils.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.util
2 |
3 | import org.joda.time.format.ISODateTimeFormat
4 | import org.joda.time.{DateTime, DateTimeComparator, DateTimeZone}
5 |
6 |
7 | object DateUtils {
8 | val ZeroTime = dtFromUtcSeconds(0)
9 |
10 | private val iso8601format = ISODateTimeFormat.dateTimeNoMillis()
11 | private val iso8601withMillis = ISODateTimeFormat.dateTime()
12 | private val dateComparator = DateTimeComparator.getInstance()
13 |
14 | def iso8601(dt: DateTime, fractions: Boolean = false): String =
15 | if (fractions) iso8601withMillis.print(dt) else iso8601format.print(dt)
16 |
17 | @inline def dtFromUtcSeconds(seconds: Int): DateTime = new DateTime(seconds * 1000L, DateTimeZone.UTC)
18 |
19 | @inline def dtFromIso8601(isoString: String): DateTime = new DateTime(isoString, DateTimeZone.UTC)
20 |
21 | /**
22 | * Implicit conversions so we can use Scala comparison operators
23 | * with JodaTime's DateTime
24 | */
25 | implicit def dateTimeToScalaWrapper(dt: DateTime): DateTimeWrapper = new DateTimeWrapper(dt)
26 |
27 | class DateTimeWrapper(dt: DateTime) extends Ordered[DateTime] with Ordering[DateTime] {
28 | def compare(that: DateTime): Int = dateComparator.compare(dt, that)
29 | def compare(a: DateTime, b: DateTime): Int = dateComparator.compare(a, b)
30 | }
31 | }
32 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/util/GuavaCacheUtils.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.util
2 |
3 | import java.util.concurrent.TimeUnit
4 |
5 | import com.google.common.cache.{CacheBuilder, Cache => GuavaCache}
6 | import com.typesafe.config.Config
7 | import ooyala.common.akka.metrics.MetricsWrapper
8 |
9 | object GuavaCacheUtils {
10 | def buildCache[K,V](cacheSize: Int, ttlSeconds: Long): GuavaCache[K,V] = {
11 | CacheBuilder.newBuilder
12 | .maximumSize(cacheSize)
13 | .expireAfterAccess(ttlSeconds, TimeUnit.SECONDS)
14 | .recordStats()
15 | .build()
16 | .asInstanceOf[GuavaCache[K,V]]
17 | }
18 |
19 | def buildCache[K,V](config: Config): GuavaCache[K,V] =
20 | buildCache(
21 | config.getInt("spark.jobserver.job-result-cache-size"),
22 | config.getLong("spark.jobserver.job-result-cache-ttl-seconds")
23 | )
24 |
25 | //pimp my cache
26 | class WrappedCache[K,V](cache: GuavaCache[K,V]) {
27 | def withMetrics(klass: Class[_]): GuavaCache[K,V] = {
28 | MetricsWrapper.newGauge(klass, "cache-hit", cache.stats.hitCount())
29 | MetricsWrapper.newGauge(klass, "cache-miss", cache.stats.missCount())
30 | MetricsWrapper.newGauge(klass, "cache-eviction", cache.stats.evictionCount())
31 | MetricsWrapper.newGauge(klass, "cache-request", cache.stats.requestCount())
32 | cache
33 | }
34 | }
35 |
36 | implicit def toWrappedCache[K,V](cache: GuavaCache[K,V]): WrappedCache[K,V] = new WrappedCache[K,V](cache)
37 | }
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/util/LRUCache.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.util
2 |
3 | import java.util.Map.Entry
4 |
5 | import ooyala.common.akka.metrics.MetricsWrapper
6 |
7 | import scala.collection.JavaConversions._
8 |
9 | /**
10 | * A convenience class to define a Least-Recently-Used Cache with a maximum size.
11 | * The oldest entries by time of last access will be removed when the number of entries exceeds
12 | * cacheSize.
13 | * For definitions of cacheSize and loadingFactor, see the docs for java.util.LinkedHashMap
14 | * @see LinkedHashMap
15 | */
16 | class LRUCache[K, V](cacheHolderClass: Class[_],
17 | cacheSize: Int,
18 | loadingFactor: Float = 0.75F) {
19 | private val cache = {
20 | val initialCapacity = math.ceil(cacheSize / loadingFactor).toInt + 1
21 | new java.util.LinkedHashMap[K, V](initialCapacity, loadingFactor, true) {
22 | protected override def removeEldestEntry(p1: Entry[K, V]): Boolean = size() > cacheSize
23 | }
24 | }
25 |
26 | // metrics for the LRU cache hits and misses.
27 | private val metricHit = MetricsWrapper.newCounter(cacheHolderClass, "cache-hit")
28 | private val metricMiss = MetricsWrapper.newCounter(cacheHolderClass, "cache-miss")
29 |
30 | /** size of the cache. This is an exact number and runs in constant time */
31 | def size: Int = cache.size()
32 |
33 | /** @return TRUE if the cache contains the key */
34 | def containsKey(k: K): Boolean = cache.get(k) match {
35 | case null =>
36 | metricMiss.inc
37 | false
38 | case _ =>
39 | metricHit.inc
40 | true
41 | }
42 |
43 | /** @return the value in cache or load a new value into cache */
44 | def get(k: K, v: => V): V = {
45 | cache.get(k) match {
46 | case null =>
47 | val evaluatedV = v
48 | cache.put(k, evaluatedV)
49 | metricMiss.inc
50 | evaluatedV
51 | case vv =>
52 | metricHit.inc
53 | vv
54 | }
55 | }
56 |
57 | def put(k: K, v: V): V = cache.put(k, v)
58 |
59 | def get(k: K): Option[V] = cache.get(k) match {
60 | case null =>
61 | metricMiss.inc
62 | None
63 | case vv =>
64 | metricHit.inc
65 | Some(vv)
66 | }
67 |
68 | def toMap = mapAsScalaMap(cache).toMap
69 | }
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/util/SparkContextFactory.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.util
2 |
3 | import com.typesafe.config.Config
4 | import org.apache.spark.SparkContext
5 |
6 | /**
7 | * I know how to make SparkContexts!
8 | * My implementing classes can be dynamically loaded using classloaders to ensure that the entire
9 | * SparkContext has access to certain dynamically loaded classes, for example, job jars.
10 | */
11 | trait SparkContextFactory {
12 | /**
13 | * Creates a SparkContext.
14 | * @param config the overall system / job server Typesafe Config
15 | * @param contextConfig the config specific to this particular job
16 | * @param contextName the name of the context to start
17 | */
18 | def makeContext(config: Config, contextConfig: Config, contextName: String): SparkContext
19 | }
20 |
21 | /**
22 | * The default factory creates a standard SparkContext.
23 | * In the future if we want to add additional methods, etc. then we can have additional factories.
24 | * For example a specialized SparkContext to manage RDDs in a user-defined way.
25 | *
26 | * If you create your own SparkContextFactory, please make sure it has zero constructor args.
27 | */
28 | class DefaultSparkContextFactory extends SparkContextFactory {
29 | import SparkJobUtils._
30 |
31 | def makeContext(config: Config, contextConfig: Config, contextName: String): SparkContext = {
32 | val conf = configToSparkConf(config, contextConfig, contextName)
33 |
34 | new SparkContext(conf)
35 | }
36 | }
37 |
--------------------------------------------------------------------------------
/job-server/src/spark.jobserver/util/SparkJobUtils.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.util
2 |
3 | import com.typesafe.config.Config
4 | import org.apache.spark.SparkConf
5 | import scala.util.Try
6 |
7 | /**
8 | * Holds a few functions common to Job Server SparkJob's and SparkContext's
9 | */
10 | object SparkJobUtils {
11 | import collection.JavaConverters._
12 |
13 | /**
14 | * Creates a SparkConf for initializing a SparkContext based on various configs.
15 | * Note that anything in contextConfig with keys beginning with spark. get
16 | * put directly in the SparkConf.
17 | *
18 | * @param config the overall Job Server configuration (Typesafe Config)
19 | * @param contextConfig the Typesafe Config specific to initializing this context
20 | * (typically based on particular context/job)
21 | * @param contextName the context name
22 | * @return a SparkConf with everything properly configured
23 | */
24 | def configToSparkConf(config: Config, contextConfig: Config,
25 | contextName: String): SparkConf = {
26 | val conf = new SparkConf()
27 | conf.setMaster(config.getString("spark.master"))
28 | .setAppName(contextName)
29 |
30 | for (cores <- Try(contextConfig.getInt("num-cpu-cores"))) {
31 | conf.set("spark.cores.max", cores.toString)
32 | }
33 | // Should be a -Xmx style string eg "512m", "1G"
34 | for (nodeMemStr <- Try(contextConfig.getString("memory-per-node"))) {
35 | conf.set("spark.executor.memory", nodeMemStr)
36 | }
37 |
38 | Try(config.getString("spark.home")).foreach { home => conf.setSparkHome(home) }
39 |
40 | // Set the Jetty port to 0 to find a random port
41 | conf.set("spark.ui.port", "0")
42 |
43 | // Set number of akka threads
44 | // TODO: need to figure out how many extra threads spark needs, besides the job threads
45 | conf.set("spark.akka.threads", (getMaxRunningJobs(config) + 4).toString)
46 |
47 | // Set any other settings in context config that start with "spark"
48 | for (e <- contextConfig.entrySet().asScala if e.getKey.startsWith("spark.")) {
49 | conf.set(e.getKey, e.getValue.unwrapped.toString)
50 | }
51 |
52 | conf
53 | }
54 |
55 | /**
56 | * Returns the maximum number of jobs that can run at the same time
57 | */
58 | def getMaxRunningJobs(config: Config): Int = {
59 | val cpuCores = Runtime.getRuntime.availableProcessors
60 | Try(config.getInt("spark.jobserver.max-jobs-per-context")).getOrElse(cpuCores)
61 | }
62 | }
63 |
--------------------------------------------------------------------------------
/job-server/src/test/resources/local.test.jobcachedao.conf:
--------------------------------------------------------------------------------
1 | # This file contains test settings for: JobSqlDAOSpec
2 |
3 | spark.jobserver {
4 | # Number of job results to keep per JobResultActor/context
5 | job-result-cache-size = 5000
6 |
7 | job-result-cache-ttl-seconds = 600
8 |
9 | jobdao = spark.jobserver.io.JobCacheDAO
10 |
11 | cachedao {
12 | rootdir = /tmp/spark-jobserver-test/cachedao/data
13 | }
14 | }
15 |
--------------------------------------------------------------------------------
/job-server/src/test/resources/local.test.jobfiledao.conf:
--------------------------------------------------------------------------------
1 | # This file contains test settings for: JobSqlDAOSpec
2 |
3 | spark.jobserver {
4 | # Number of job results to keep per JobResultActor/context
5 | job-result-cache-size = 5000
6 |
7 | job-result-cache-ttl-seconds = 600
8 |
9 | jobdao = spark.jobserver.io.JobFileDAO
10 |
11 | filedao {
12 | rootdir = /tmp/spark-jobserver-test/filedao/data
13 | }
14 | }
15 |
--------------------------------------------------------------------------------
/job-server/src/test/resources/local.test.jobsqldao.conf:
--------------------------------------------------------------------------------
1 | # This file contains test settings for: JobSqlDAOSpec
2 |
3 | spark.jobserver {
4 | jobdao = spark.jobserver.io.JobSqlDAO
5 |
6 | sqldao {
7 | rootdir = /tmp/spark-job-server-test/sqldao/data
8 | # https://coderwall.com/p/a2vnxg
9 | h2 {
10 | url = "jdbc:h2:mem:jobserver-test;DATABASE_TO_UPPER=false;DB_CLOSE_DELAY=-1"
11 | }
12 | }
13 | }
14 |
--------------------------------------------------------------------------------
/job-server/src/test/resources/log4j.properties:
--------------------------------------------------------------------------------
1 | # Default LOG4J properties logs to a file and overwrites it every time. Works well for tests.
2 |
3 | # Root logger option
4 | log4j.rootLogger=INFO, LOGFILE
5 |
6 | log4j.appender.LOGFILE=org.apache.log4j.FileAppender
7 | log4j.appender.LOGFILE.File=job-server-test.log
8 | log4j.appender.LOGFILE.layout=org.apache.log4j.PatternLayout
9 | # log4j.appender.LOGFILE.layout.ConversionPattern=%d %-5p %c - %m%n
10 | log4j.appender.LOGFILE.layout.ConversionPattern=[%d] %-5p %.26c [%X{testName}] [%X{akkaSource}] - %m%n
11 | log4j.appender.LOGFILE.Append=false
12 |
--------------------------------------------------------------------------------
/job-server/src/test/resources/logback-test.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 |
6 | job-server-test.log
7 |
8 |
9 | false
10 |
11 |
12 | [%date{ISO8601}] %-5level %logger{26} [%X{akkaSource}] - %msg%n
13 |
14 |
15 |
16 |
17 |
18 |
19 |
20 |
21 |
--------------------------------------------------------------------------------
/job-server/test/spark.jobserver/InMemoryDAO.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import com.typesafe.config.Config
4 | import java.io.{BufferedOutputStream, FileOutputStream}
5 | import org.joda.time.DateTime
6 | import scala.collection.mutable
7 | import spark.jobserver.io.{JobDAO, JobInfo}
8 |
9 | /**
10 | * In-memory DAO for easy unit testing
11 | */
12 | class InMemoryDAO extends JobDAO {
13 | val jars = mutable.HashMap.empty[(String, DateTime), Array[Byte]]
14 |
15 | def saveJar(appName: String, uploadTime: DateTime, jarBytes: Array[Byte]) {
16 | jars((appName, uploadTime)) = jarBytes
17 | }
18 |
19 | def getApps(): Map[String, DateTime] = {
20 | jars.keys
21 | .groupBy(_._1)
22 | .map { case (appName, appUploadTimeTuples) =>
23 | appName -> appUploadTimeTuples.map(_._2).toSeq.head
24 | }.toMap
25 | }
26 |
27 | def retrieveJarFile(appName: String, uploadTime: DateTime): String = {
28 | // Write the jar bytes to a temporary file
29 | val outFile = java.io.File.createTempFile("InMemoryDAO", ".jar")
30 | outFile.deleteOnExit()
31 | val bos = new BufferedOutputStream(new FileOutputStream(outFile))
32 | try {
33 | bos.write(jars((appName, uploadTime)))
34 | } finally {
35 | bos.close()
36 | }
37 | outFile.getAbsolutePath()
38 | }
39 |
40 | val jobInfos = mutable.HashMap.empty[String, JobInfo]
41 |
42 | def saveJobInfo(jobInfo: JobInfo) { jobInfos(jobInfo.jobId) = jobInfo }
43 |
44 | def getJobInfos(): Map[String, JobInfo] = jobInfos.toMap
45 |
46 | override def getJobInfosLimit(limit: Int): Map[String, JobInfo] = jobInfos.takeRight(limit).toMap
47 |
48 | override def getJobInfo(jobId: String): Option[JobInfo] = jobInfos.get(jobId)
49 |
50 | val jobConfigs = mutable.HashMap.empty[String, Config]
51 |
52 | def saveJobConfig(jobId: String, jobConfig: Config) { jobConfigs(jobId) = jobConfig }
53 |
54 | def getJobConfigs(): Map[String, Config] = jobConfigs.toMap
55 | }
56 |
--------------------------------------------------------------------------------
/job-server/test/spark.jobserver/JobInfoActorSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.{Props, ActorRef, ActorSystem}
4 | import akka.testkit.{TestKit, ImplicitSender}
5 | import org.scalatest.{FunSpec, BeforeAndAfter, BeforeAndAfterAll}
6 | import org.scalatest.matchers.ShouldMatchers
7 | import spark.jobserver.io.JobDAO
8 |
9 | object JobInfoActorSpec {
10 | val system = ActorSystem("test")
11 | }
12 |
13 | class JobInfoActorSpec extends TestKit(JobInfoActorSpec.system) with ImplicitSender
14 | with FunSpec with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll {
15 |
16 | import com.typesafe.config._
17 | import CommonMessages.NoSuchJobId
18 | import JobInfoActor._
19 |
20 | private val jobId = "jobId"
21 | private val jobConfig = ConfigFactory.empty()
22 |
23 | override def afterAll() {
24 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(JobInfoActorSpec.system)
25 | }
26 |
27 | var actor: ActorRef = _
28 | var dao: JobDAO = _
29 |
30 | before {
31 | dao = new InMemoryDAO
32 | actor = system.actorOf(Props(classOf[JobInfoActor], dao, system.actorOf(Props(classOf[LocalContextSupervisorActor], dao))))
33 | }
34 |
35 | after {
36 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(actor)
37 | }
38 |
39 | describe("JobInfoActor") {
40 | it("should store a job configuration") {
41 | actor ! StoreJobConfig(jobId, jobConfig)
42 | expectMsg(JobConfigStored)
43 | dao.getJobConfigs.get(jobId) should be (Some(jobConfig))
44 | }
45 |
46 | it("should return a job configuration when the jobId exists") {
47 | actor ! StoreJobConfig(jobId, jobConfig)
48 | expectMsg(JobConfigStored)
49 | actor ! GetJobConfig(jobId)
50 | expectMsg(jobConfig)
51 | }
52 |
53 | it("should return error if jobId does not exist") {
54 | actor ! GetJobConfig(jobId)
55 | expectMsg(NoSuchJobId)
56 | }
57 | }
58 | }
59 |
--------------------------------------------------------------------------------
/job-server/test/spark.jobserver/JobManagerActorAdHocSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.Props
4 |
5 | /**
6 | * This tests JobManagerActor of AdHoc context.
7 | * Pass true to isAdHoc when the JobManagerActor is created.
8 | */
9 | class JobManagerActorAdHocSpec extends JobManagerSpec {
10 |
11 | before {
12 | dao = new InMemoryDAO
13 | manager =
14 | system.actorOf(JobManagerActor.props(dao, "test", JobManagerSpec.config, true))
15 | }
16 |
17 | }
18 |
--------------------------------------------------------------------------------
/job-server/test/spark.jobserver/JobManagerActorSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.Props
4 | import com.codahale.metrics.Counter
5 | import ooyala.common.akka.metrics.MetricsWrapper
6 | import spark.jobserver.CommonMessages.{JobErroredOut, JobResult}
7 |
8 | class JobManagerActorSpec extends JobManagerSpec {
9 | import scala.concurrent.duration._
10 |
11 | // Metrics for the job cache
12 | private val metricCacheHitName = "spark.jobserver.JobCache.cache-hit"
13 | private val metricCacheMissName = "spark.jobserver.JobCache.cache-miss"
14 | var metricCacheHit: Counter = _
15 | var metricCacheMiss: Counter = _
16 |
17 | before {
18 | // Resets the metrics for the job cache
19 | MetricsWrapper.getRegistry.remove(metricCacheHitName)
20 | MetricsWrapper.getRegistry.remove(metricCacheMissName)
21 | MetricsWrapper.getRegistry.counter(metricCacheHitName)
22 | MetricsWrapper.getRegistry.counter(metricCacheMissName)
23 |
24 | val counters = MetricsWrapper.getRegistry.getCounters
25 | metricCacheHit = counters.get(metricCacheHitName)
26 | metricCacheMiss = counters.get(metricCacheMissName)
27 |
28 | dao = new InMemoryDAO
29 | manager =
30 | system.actorOf(JobManagerActor.props(dao, "test", JobManagerSpec.config, false))
31 | }
32 |
33 | describe("starting jobs") {
34 | it("jobs should be able to cache RDDs and retrieve them through getPersistentRDDs") {
35 | manager ! JobManagerActor.Initialize
36 | expectMsgClass(classOf[JobManagerActor.Initialized])
37 |
38 | uploadTestJar()
39 | manager ! JobManagerActor.StartJob("demo", classPrefix + "CacheSomethingJob", emptyConfig,
40 | errorEvents ++ syncEvents)
41 | val JobResult(_, sum: Int) = expectMsgClass(classOf[JobResult])
42 |
43 | manager ! JobManagerActor.StartJob("demo", classPrefix + "AccessCacheJob", emptyConfig,
44 | errorEvents ++ syncEvents)
45 | val JobResult(_, sum2: Int) = expectMsgClass(classOf[JobResult])
46 |
47 | sum2 should equal (sum)
48 | }
49 |
50 | it ("jobs should be able to cache and retrieve RDDs by name") {
51 | manager ! JobManagerActor.Initialize
52 | expectMsgClass(classOf[JobManagerActor.Initialized])
53 |
54 | uploadTestJar()
55 | manager ! JobManagerActor.StartJob("demo", classPrefix + "CacheRddByNameJob", emptyConfig,
56 | errorEvents ++ syncEvents)
57 | expectMsgPF(1 second, "Expected a JobResult or JobErroredOut message!") {
58 | case JobResult(_, sum: Int) => sum should equal (1 + 4 + 9 + 16 + 25)
59 | case JobErroredOut(_, _, error: Throwable) => throw error
60 | }
61 | }
62 | }
63 |
64 | describe("JobManagerActor JobCache") {
65 | it("should increase job cache hit count") {
66 | metricCacheHit.getCount should equal(0)
67 | metricCacheMiss.getCount should equal(0)
68 |
69 | manager ! JobManagerActor.Initialize
70 | expectMsgClass(classOf[JobManagerActor.Initialized])
71 |
72 | uploadTestJar()
73 |
74 | manager ! JobManagerActor.StartJob("demo", classPrefix + "SimpleObjectJob", emptyConfig,
75 | errorEvents ++ syncEvents)
76 | expectMsgClass(classOf[JobResult])
77 | metricCacheHit.getCount should equal(0)
78 | // Demo for the first time, misses the job cache
79 | metricCacheMiss.getCount should equal(1)
80 |
81 | manager ! JobManagerActor.StartJob("demo", classPrefix + "SimpleObjectJob", emptyConfig,
82 | errorEvents ++ syncEvents)
83 | expectMsgClass(classOf[JobResult])
84 | // Demo for the second time, hits the job cache
85 | metricCacheHit.getCount should equal(1)
86 | metricCacheMiss.getCount should equal(1)
87 | }
88 |
89 | it("should increase job cache miss count") {
90 | metricCacheHit.getCount should equal(0)
91 | metricCacheMiss.getCount should equal(0)
92 |
93 | manager ! JobManagerActor.Initialize
94 | expectMsgClass(classOf[JobManagerActor.Initialized])
95 |
96 | uploadTestJar()
97 |
98 | manager ! JobManagerActor.StartJob("demo", classPrefix + "SimpleObjectJob", emptyConfig,
99 | errorEvents ++ syncEvents)
100 | expectMsgClass(classOf[JobResult])
101 | metricCacheHit.getCount should equal(0)
102 | // Demo for the first time, misses the job cache
103 | metricCacheMiss.getCount should equal(1)
104 |
105 | uploadTestJar("new demo")
106 |
107 | manager ! JobManagerActor.StartJob("new demo", classPrefix + "SimpleObjectJob", emptyConfig,
108 | errorEvents ++ syncEvents)
109 | expectMsgClass(classOf[JobResult])
110 | metricCacheHit.getCount should equal(0)
111 | // New demo for the first time, misses the job cache again
112 | metricCacheMiss.getCount should equal(2)
113 | }
114 | }
115 | }
116 |
--------------------------------------------------------------------------------
/job-server/test/spark.jobserver/JobResultActorSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.{ActorRef, ActorSystem}
4 | import akka.testkit.{ImplicitSender, TestActorRef, TestKit}
5 | import org.scalatest.matchers.ShouldMatchers
6 | import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSpec}
7 |
8 |
9 | object JobResultActorSpec {
10 | val system = ActorSystem("job-result-actor-test")
11 | }
12 |
13 | class JobResultActorSpec extends TestKit(JobResultActorSpec.system) with ImplicitSender
14 | with FunSpec with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll {
15 |
16 | import CommonMessages._
17 |
18 | override def afterAll() {
19 | TestKit.shutdownActorSystem(JobResultActorSpec.system)
20 | }
21 |
22 | describe("JobResultActor") {
23 | it("should return error if non-existing jobs are asked") {
24 | withActor { actor =>
25 | actor ! GetJobResult("jobId")
26 | expectMsg(NoSuchJobId)
27 | }
28 | }
29 |
30 | it("should get back existing result") {
31 | withActor { actor =>
32 | actor ! JobResult("jobId", 10)
33 | actor ! GetJobResult("jobId")
34 | expectMsg(JobResult("jobId", 10))
35 | }
36 | }
37 |
38 | it("should be informed only once by subscribed result") {
39 | withActor { actor =>
40 | actor ! Subscribe("jobId", self, Set(classOf[JobResult]))
41 | actor ! JobResult("jobId", 10)
42 | expectMsg(JobResult("jobId", 10))
43 |
44 | actor ! JobResult("jobId", 20)
45 | expectNoMsg() // shouldn't get it again
46 | }
47 | }
48 |
49 | it("should not be informed unsubscribed result") {
50 | withActor { actor =>
51 | actor ! Subscribe("jobId", self, Set(classOf[JobResult]))
52 | actor ! Unsubscribe("jobId", self)
53 | actor ! JobResult("jobId", 10)
54 | expectNoMsg()
55 | }
56 | }
57 |
58 | it("should not publish if do not subscribe to JobResult events") {
59 | withActor { actor =>
60 | actor ! Subscribe("jobId", self, Set(classOf[JobValidationFailed], classOf[JobErroredOut]))
61 | actor ! JobResult("jobId", 10)
62 | expectNoMsg()
63 | }
64 | }
65 |
66 | it("should return error if non-existing subscription is unsubscribed") {
67 | withActor{ actor =>
68 | actor ! Unsubscribe("jobId", self)
69 | expectMsg(NoSuchJobId)
70 | }
71 | }
72 | }
73 |
74 | def withActor[T](f: ActorRef => T): T ={
75 | val actor = TestActorRef(new JobResultActor)
76 | try {
77 | f(actor)
78 | } finally {
79 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(actor)
80 | }
81 | }
82 | }
--------------------------------------------------------------------------------
/job-server/test/spark.jobserver/JobResultCacheSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.{ActorRef, ActorSystem, Props}
4 | import akka.testkit.{ImplicitSender, TestActorRef, TestKit}
5 | import com.typesafe.config.ConfigFactory
6 | import ooyala.common.akka.metrics.MetricsWrapper
7 | import org.scalatest.matchers.ShouldMatchers
8 | import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSpec}
9 |
10 |
11 | object JobResultCacheSpec {
12 | val config = ConfigFactory.parseString("""
13 | spark {
14 | jobserver.job-result-cache-size = 10
15 | jobserver.job-result-cache-ttl-seconds = 1
16 | }
17 | """)
18 |
19 | val system = ActorSystem("job-result-cache-test", config)
20 | }
21 |
22 | class JobResultCacheSpec extends TestKit(JobResultCacheSpec.system) with ImplicitSender
23 | with FunSpec with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll {
24 |
25 | import CommonMessages._
26 |
27 | override def afterAll() {
28 | TestKit.shutdownActorSystem(JobResultCacheSpec.system)
29 | }
30 |
31 | // Metrics for job result cache
32 | private val metricCacheHitName = "spark.jobserver.JobResultActor.cache-hit"
33 | private val metricCacheMissName = "spark.jobserver.JobResultActor.cache-miss"
34 | private val metricCacheRequestName = "spark.jobserver.JobResultActor.cache-request"
35 | private val metricCacheEvictionName = "spark.jobserver.JobResultActor.cache-eviction"
36 |
37 | before {
38 | MetricsWrapper.getRegistry.remove(metricCacheHitName)
39 | MetricsWrapper.getRegistry.remove(metricCacheMissName)
40 | MetricsWrapper.getRegistry.remove(metricCacheRequestName)
41 | MetricsWrapper.getRegistry.remove(metricCacheEvictionName)
42 | }
43 |
44 | describe("JobResultActor cache") {
45 | it("should increase cache hit count") {
46 | withActor { actor =>
47 |
48 | actor ! JobResult("jobId", 10)
49 | actor ! GetJobResult("jobId")
50 | expectMsg(JobResult("jobId", 10))
51 | // Hits the job result cache for the first time
52 | MetricsWrapper.getRegistry.getGauges.get(metricCacheHitName).getValue should equal(1)
53 | MetricsWrapper.getRegistry.getGauges.get(metricCacheMissName).getValue should equal(0)
54 | MetricsWrapper.getRegistry.getGauges.get(metricCacheRequestName).getValue should equal(1)
55 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(0)
56 |
57 | actor ! GetJobResult("jobId")
58 | expectMsg(JobResult("jobId", 10))
59 | // Hits the job result cache for the second time
60 | MetricsWrapper.getRegistry.getGauges.get(metricCacheHitName).getValue should equal(2)
61 | MetricsWrapper.getRegistry.getGauges.get(metricCacheMissName).getValue should equal(0)
62 | MetricsWrapper.getRegistry.getGauges.get(metricCacheRequestName).getValue should equal(2)
63 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(0)
64 | }
65 | }
66 |
67 | it("should increase cache miss count") {
68 | withActor { actor =>
69 | actor ! JobResult("jobId", 10)
70 | actor ! GetJobResult("NoJobId")
71 | expectMsg(NoSuchJobId)
72 | MetricsWrapper.getRegistry.getGauges.get(metricCacheHitName).getValue should equal(0)
73 | MetricsWrapper.getRegistry.getGauges.get(metricCacheMissName).getValue should equal(1)
74 | MetricsWrapper.getRegistry.getGauges.get(metricCacheRequestName).getValue should equal(1)
75 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(0)
76 |
77 | actor ! GetJobResult("NoJobId")
78 | expectMsg(NoSuchJobId)
79 | MetricsWrapper.getRegistry.getGauges.get(metricCacheHitName).getValue should equal(0)
80 | MetricsWrapper.getRegistry.getGauges.get(metricCacheMissName).getValue should equal(2)
81 | MetricsWrapper.getRegistry.getGauges.get(metricCacheRequestName).getValue should equal(2)
82 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(0)
83 | }
84 | }
85 |
86 | it("should evict entries according to configured TTL") {
87 | withActor { actor =>
88 | val jobResultActor = system.actorOf(Props[JobResultActor])
89 |
90 | (1 to 10).foreach { i =>
91 | actor ! JobResult(s"jobId_$i", 10)
92 | }
93 |
94 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(0)
95 |
96 | //from javadoc: requested entries may be evicted on each cache modification, on occasional
97 | //cache accesses, or on calls to Cache#cleanUp
98 | Thread.sleep(1500)
99 | actor ! JobResult("jobId", 10)
100 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(10)
101 | actor ! GetJobResult("jobId_1")
102 | MetricsWrapper.getRegistry.getGauges.get(metricCacheMissName).getValue should equal(1)
103 | }
104 | }
105 |
106 | it("should evict entries according to configured cache size") {
107 | withActor { actor =>
108 | (1 to 10).foreach { i =>
109 | actor ! JobResult(s"jobId_$i", 10)
110 | }
111 |
112 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(0)
113 |
114 | actor ! JobResult("jobId_X", 10)
115 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(1)
116 |
117 | actor ! JobResult("jobId_Y", 10)
118 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(2)
119 | }
120 | }
121 | }
122 |
123 | def withActor[T](f: ActorRef => T): T ={
124 | val actor = TestActorRef(new JobResultActor)
125 | try {
126 | f(actor)
127 | } finally {
128 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(actor)
129 | }
130 | }
131 | }
132 |
--------------------------------------------------------------------------------
/job-server/test/spark.jobserver/JobStatusActorSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.{Props, PoisonPill, ActorRef, ActorSystem}
4 | import akka.testkit.{TestKit, ImplicitSender}
5 | import spark.jobserver.io.{JarInfo, JobInfo, JobDAO}
6 | import org.joda.time.DateTime
7 | import org.scalatest.matchers.ShouldMatchers
8 | import org.scalatest.{FunSpec, BeforeAndAfter, BeforeAndAfterAll}
9 |
10 | object JobStatusActorSpec {
11 | val system = ActorSystem("test")
12 | }
13 |
14 | class JobStatusActorSpec extends TestKit(JobStatusActorSpec.system) with ImplicitSender
15 | with FunSpec with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll {
16 |
17 | import com.typesafe.config._
18 | import CommonMessages._
19 | import JobStatusActor._
20 |
21 | private val jobId = "jobId"
22 | private val contextName = "contextName"
23 | private val appName = "appName"
24 | private val jarInfo = JarInfo(appName, DateTime.now)
25 | private val classPath = "classPath"
26 | private val jobInfo = JobInfo(jobId, contextName, jarInfo, classPath, DateTime.now, None, None)
27 | private val jobConfig = ConfigFactory.empty()
28 |
29 | override def afterAll() {
30 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(JobStatusActorSpec.system)
31 | }
32 |
33 | var actor: ActorRef = _
34 | var receiver: ActorRef = _
35 | var dao: JobDAO = _
36 |
37 | before {
38 | dao = new InMemoryDAO
39 | actor = system.actorOf(Props(classOf[JobStatusActor], dao))
40 | receiver = system.actorOf(Props[JobResultActor])
41 | }
42 |
43 | after {
44 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(actor)
45 | }
46 |
47 | describe("JobStatusActor") {
48 | it("should return empty sequence if there is no job infos") {
49 | actor ! GetRunningJobStatus
50 | expectMsg(Seq.empty)
51 | }
52 |
53 | it("should return error if non-existing job is unsubscribed") {
54 | actor ! Unsubscribe(jobId, receiver)
55 | expectMsg(NoSuchJobId)
56 | }
57 |
58 | it("should not initialize a job more than two times") {
59 | actor ! JobInit(jobInfo)
60 | actor ! JobInit(jobInfo)
61 | expectMsg(JobInitAlready)
62 | }
63 |
64 | it("should be informed JobStarted until it is unsubscribed") {
65 | actor ! JobInit(jobInfo)
66 | actor ! Subscribe(jobId, self, Set(classOf[JobStarted]))
67 | val msg = JobStarted(jobId, contextName, DateTime.now)
68 | actor ! msg
69 | expectMsg(msg)
70 |
71 | actor ! msg
72 | expectMsg(msg)
73 |
74 | actor ! Unsubscribe(jobId, self)
75 | actor ! JobStarted(jobId, contextName, DateTime.now)
76 | expectNoMsg() // shouldn't get it again
77 |
78 | actor ! Unsubscribe(jobId, self)
79 | expectMsg(NoSuchJobId)
80 | }
81 |
82 | it("should be ok to subscribe beofore job init") {
83 | actor ! Subscribe(jobId, self, Set(classOf[JobStarted]))
84 | actor ! JobInit(jobInfo)
85 | val msg = JobStarted(jobId, contextName, DateTime.now)
86 | actor ! msg
87 | expectMsg(msg)
88 | }
89 |
90 | it("should be informed JobValidationFailed once") {
91 | actor ! JobInit(jobInfo)
92 | actor ! Subscribe(jobId, self, Set(classOf[JobValidationFailed]))
93 | val msg = JobValidationFailed(jobId, DateTime.now, new Throwable)
94 | actor ! msg
95 | expectMsg(msg)
96 |
97 | actor ! msg
98 | expectMsg(NoSuchJobId)
99 | }
100 |
101 | it("should be informed JobFinished until it is unsubscribed") {
102 | actor ! JobInit(jobInfo)
103 | actor ! JobStarted(jobId, contextName, DateTime.now)
104 | actor ! Subscribe(jobId, self, Set(classOf[JobFinished]))
105 | val msg = JobFinished(jobId, DateTime.now)
106 | actor ! msg
107 | expectMsg(msg)
108 |
109 | actor ! msg
110 | expectMsg(NoSuchJobId)
111 | }
112 |
113 | it("should be informed JobErroredOut until it is unsubscribed") {
114 | actor ! JobInit(jobInfo)
115 | actor ! JobStarted(jobId, contextName, DateTime.now)
116 | actor ! Subscribe(jobId, self, Set(classOf[JobErroredOut]))
117 | val msg = JobErroredOut(jobId, DateTime.now, new Throwable)
118 | actor ! msg
119 | expectMsg(msg)
120 |
121 | actor ! msg
122 | expectMsg(NoSuchJobId)
123 | }
124 |
125 | it("should update status correctly") {
126 | actor ! JobInit(jobInfo)
127 | actor ! GetRunningJobStatus
128 | expectMsg(Seq(jobInfo))
129 |
130 | val startTime = DateTime.now
131 | actor ! JobStarted(jobId, contextName, startTime)
132 | actor ! GetRunningJobStatus
133 | expectMsg(Seq(JobInfo(jobId, contextName, jarInfo, classPath, startTime, None, None)))
134 |
135 | val finishTIme = DateTime.now
136 | actor ! JobFinished(jobId, finishTIme)
137 | actor ! GetRunningJobStatus
138 | expectMsg(Seq.empty)
139 | }
140 |
141 | it("should update JobValidationFailed status correctly") {
142 | val initTime = DateTime.now
143 | val jobInfo = JobInfo(jobId, contextName, jarInfo, classPath, initTime, None, None)
144 | actor ! JobInit(jobInfo)
145 |
146 | val failedTime = DateTime.now
147 | val err = new Throwable
148 | actor ! JobValidationFailed(jobId, failedTime, err)
149 | actor ! GetRunningJobStatus
150 | expectMsg(Seq.empty)
151 | }
152 |
153 | it("should update JobErroredOut status correctly") {
154 | actor ! JobInit(jobInfo)
155 |
156 | val startTime = DateTime.now
157 | actor ! JobStarted(jobId, contextName, startTime)
158 |
159 | val failedTime = DateTime.now
160 | val err = new Throwable
161 | actor ! JobErroredOut(jobId, failedTime, err)
162 | actor ! GetRunningJobStatus
163 | expectMsg(Seq.empty)
164 | }
165 | }
166 | }
167 |
--------------------------------------------------------------------------------
/job-server/test/spark.jobserver/LocalContextSupervisorHaSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.{ActorRef, Props, ActorSystem}
4 | import akka.testkit.{ImplicitSender, TestKit}
5 | import akka.util.Timeout
6 | import com.typesafe.config.ConfigFactory
7 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSpec}
8 | import org.scalatest.matchers.ShouldMatchers
9 | import scala.concurrent.duration._
10 | import spark.jobserver.io.JobSqlDAO
11 |
12 |
13 | class LocalContextSupervisorHaSpec(system: ActorSystem) extends TestKit(system) with ImplicitSender
14 | with FunSpec with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll {
15 |
16 | def this() = this(ActorSystem("test", LocalContextSupervisorSpec.config))
17 |
18 | override def afterAll() {
19 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(system)
20 | }
21 |
22 | // Expects the response to GetContext message for the named context.
23 | private def expectResponseToGetContext(contextName: String) = {
24 | expectMsgPF() {
25 | case (manager: ActorRef, resultActor: ActorRef) =>
26 | manager.path.name should equal(contextName)
27 | resultActor.path.name should equal("result-actor")
28 | }
29 | }
30 |
31 | var supervisor_a: ActorRef = _
32 | var supervisor_b: ActorRef = _
33 | val contextConfig = LocalContextSupervisorSpec.config.getConfig("spark.context-settings")
34 |
35 | implicit val timeout: Timeout = 10 seconds
36 |
37 | // This is needed to help tests pass on some MBPs when working from home
38 | System.setProperty("spark.driver.host", "localhost")
39 |
40 | before {
41 | val sqlDaoConfigStr = "spark.jobserver.sqldao.h2.url=\"jdbc:h2:file:" +
42 | "/tmp/spark-job-server-test/sqldao/data/" + java.util.UUID.randomUUID() + "\""
43 | val sqlDaoConfig = ConfigFactory.parseString(sqlDaoConfigStr)
44 |
45 | val dao_a = new JobSqlDAO(sqlDaoConfig)
46 | supervisor_a = system.actorOf(Props(classOf[LocalContextSupervisorActor], dao_a))
47 |
48 | val dao_b = new JobSqlDAO(sqlDaoConfig)
49 | supervisor_b = system.actorOf(Props(classOf[LocalContextSupervisorActor], dao_b))
50 | }
51 |
52 | after {
53 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(supervisor_a)
54 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(supervisor_b)
55 | }
56 |
57 | import ContextSupervisor._
58 |
59 | describe("context high availability") {
60 | it("should return no such context if DAO hasn't added it") {
61 | supervisor_a ! AddContext("c1", contextConfig)
62 | expectMsg(ContextInitialized)
63 |
64 | supervisor_b ! GetContext("c2")
65 | expectMsg(NoSuchContext)
66 | }
67 |
68 | it("should allow two context supervisors to share the same context config") {
69 | supervisor_a ! AddContext("c1", contextConfig)
70 | expectMsg(ContextInitialized)
71 |
72 | supervisor_b ! GetContext("c1")
73 | expectResponseToGetContext("c1")
74 | }
75 |
76 | it("should return ContextJobDaoError if context config already added by DAO") {
77 | supervisor_a ! AddContext("c1", contextConfig)
78 | expectMsg(ContextInitialized)
79 |
80 | supervisor_b ! AddContext("c1", contextConfig)
81 | expectMsgType[ContextJobDaoError]
82 | }
83 |
84 | it("should retrieve context config from persistent storage after context stopped") {
85 | supervisor_a ! AddContext("c1", contextConfig)
86 | expectMsg(ContextInitialized)
87 |
88 | supervisor_b ! GetContext("c1")
89 | expectResponseToGetContext("c1")
90 |
91 | supervisor_b ! StopContext("c1")
92 | expectMsg(ContextStopped)
93 |
94 | Thread.sleep(2000) // wait for a while since deleting context is an asyc call
95 |
96 | supervisor_b ! GetContext("c1")
97 | expectResponseToGetContext("c1")
98 | }
99 | }
100 | }
--------------------------------------------------------------------------------
/job-server/test/spark.jobserver/LocalContextSupervisorSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor._
4 | import akka.testkit.{TestKit, ImplicitSender}
5 | import com.typesafe.config.ConfigFactory
6 | import ooyala.common.akka.metrics.MetricsWrapper
7 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter}
8 | import org.scalatest.FunSpec
9 | import org.scalatest.matchers.ShouldMatchers
10 | import spark.jobserver.CommonMessages.{Unsubscribe, JobStarted, Subscribe}
11 | import spark.jobserver.io.JobDAO
12 |
13 | object LocalContextSupervisorSpec {
14 | val config = ConfigFactory.parseString("""
15 | spark {
16 | master = "local[4]"
17 | temp-contexts {
18 | num-cpu-cores = 4 # Number of cores to allocate. Required.
19 | memory-per-node = 512m # Executor memory per node, -Xmx style eg 512m, 1G, etc.
20 | }
21 | jobserver.job-result-cache-size = 100
22 | jobserver.job-result-cache-ttl-seconds = 60
23 | jobserver.context-creation-timeout = 5 s
24 | jobserver.context-factory = spark.jobserver.util.DefaultSparkContextFactory
25 | contexts {
26 | olap-demo {
27 | num-cpu-cores = 4
28 | memory-per-node = 512m
29 | }
30 | }
31 | context-settings {
32 | num-cpu-cores = 2
33 | memory-per-node = 512m
34 | }
35 | }
36 | akka.log-dead-letters = 0
37 | """)
38 |
39 | val system = ActorSystem("test", config)
40 | }
41 |
42 | class LocalContextSupervisorSpec extends TestKit(LocalContextSupervisorSpec.system) with ImplicitSender
43 | with FunSpec with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll {
44 |
45 | override def afterAll() {
46 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(LocalContextSupervisorSpec.system)
47 | }
48 |
49 | var supervisor: ActorRef = _
50 | var dao: JobDAO = _
51 |
52 | val contextConfig = LocalContextSupervisorSpec.config.getConfig("spark.context-settings")
53 |
54 | val counterName = "spark.jobserver.LocalContextSupervisorActor.num-contexts"
55 |
56 | // This is needed to help tests pass on some MBPs when working from home
57 | System.setProperty("spark.driver.host", "localhost")
58 |
59 | before {
60 | dao = new InMemoryDAO
61 | supervisor = system.actorOf(Props(classOf[LocalContextSupervisorActor], dao))
62 | }
63 |
64 | after {
65 | val counters: java.util.SortedMap[String, com.codahale.metrics.Counter] = MetricsWrapper.registry.getCounters
66 | val counter = counters.get(counterName)
67 | counter.dec(counter.getCount)
68 |
69 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(supervisor)
70 |
71 | }
72 |
73 | import ContextSupervisor._
74 |
75 | describe("context management") {
76 | it("should list empty contexts at startup") {
77 | supervisor ! ListContexts
78 | expectMsg(Seq.empty[String])
79 | }
80 |
81 | it("can add contexts from jobConfig") {
82 | supervisor ! AddContextsFromConfig
83 | Thread sleep 2000
84 | supervisor ! ListContexts
85 | expectMsg(Seq("olap-demo"))
86 | }
87 |
88 | it("should be able to add multiple new contexts") {
89 | supervisor ! AddContext("c1", contextConfig)
90 | supervisor ! AddContext("c2", contextConfig)
91 | expectMsg(ContextInitialized)
92 | expectMsg(ContextInitialized)
93 | supervisor ! ListContexts
94 | expectMsg(Seq("c1", "c2"))
95 | supervisor ! GetResultActor("c1")
96 | val rActor = expectMsgClass(classOf[ActorRef])
97 | rActor.path.toString should endWith ("result-actor")
98 | rActor.path.toString should not include ("global")
99 | }
100 |
101 | it("should be able to stop contexts already running") {
102 | import scala.concurrent.duration._
103 | supervisor ! AddContext("c1", contextConfig)
104 | expectMsg(ContextInitialized)
105 | supervisor ! ListContexts
106 | expectMsg(Seq("c1"))
107 |
108 | supervisor ! StopContext("c1")
109 | expectMsg(ContextStopped)
110 |
111 | Thread.sleep(2000) // wait for a while since deleting context is an asyc call
112 | supervisor ! ListContexts
113 | expectMsg(Seq.empty[String])
114 | }
115 |
116 | it("should return NoSuchContext if attempt to stop nonexisting context") {
117 | supervisor ! StopContext("c1")
118 | expectMsg(NoSuchContext)
119 | }
120 |
121 | it("should not allow creation of an already existing context") {
122 | supervisor ! AddContext("c1", contextConfig)
123 | expectMsg(ContextInitialized)
124 |
125 | supervisor ! AddContext("c1", contextConfig)
126 | expectMsg(ContextAlreadyExists)
127 | }
128 |
129 | it("should inc/dec context counter correctly") {
130 | val counters = MetricsWrapper.registry.getCounters
131 | val counter = counters.get(counterName)
132 | counter should not be (null)
133 |
134 | supervisor ! AddContext("c11", contextConfig)
135 | expectMsg(ContextInitialized)
136 | supervisor ! ListContexts
137 | expectMsg(Seq("c11"))
138 |
139 | (counter.getCount) should equal (1)
140 |
141 | supervisor ! StopContext("c11")
142 | expectMsg(ContextStopped)
143 | supervisor ! ListContexts
144 | expectMsg(Seq.empty[String])
145 |
146 |
147 | (counter.getCount) should equal (0)
148 | }
149 |
150 |
151 | }
152 | }
153 |
--------------------------------------------------------------------------------
/job-server/test/spark.jobserver/NamedRddsSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.{PoisonPill, Props, ActorRef, ActorSystem}
4 | import akka.testkit.{ImplicitSender, TestKit}
5 | import org.apache.spark.SparkContext
6 | import org.scalatest.matchers.ShouldMatchers
7 | import org.scalatest.{FunSpec, BeforeAndAfterAll, BeforeAndAfter}
8 |
9 | class NamedRddsSpec extends TestKit(ActorSystem("NamedRddsSpec")) with FunSpec
10 | with ImplicitSender with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll {
11 | System.setProperty("spark.cores.max", Runtime.getRuntime.availableProcessors.toString)
12 | System.setProperty("spark.executor.memory", "512m")
13 | System.setProperty("spark.akka.threads", Runtime.getRuntime.availableProcessors.toString)
14 |
15 | // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
16 | System.clearProperty("spark.driver.port")
17 | System.clearProperty("spark.hostPort")
18 |
19 | val sc = new SparkContext("local[4]", getClass.getSimpleName)
20 | val rddManager: ActorRef = system.actorOf(Props(classOf[RddManagerActor], sc))
21 | val namedRdds: NamedRdds = new JobServerNamedRdds(rddManager)
22 |
23 | before {
24 | namedRdds.getNames.foreach { rddName => namedRdds.destroy(rddName) }
25 | }
26 |
27 | override def afterAll() {
28 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(rddManager)
29 | sc.stop()
30 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(system)
31 | }
32 |
33 | describe("NamedRdds") {
34 | it("get() should return None when RDD does not exist") {
35 | namedRdds.get[Int]("No such RDD") should equal (None)
36 | }
37 |
38 | it("get() should return Some(RDD) when it exists") {
39 | val rdd = sc.parallelize(Seq(1, 2, 3))
40 | namedRdds.update("rdd1", rdd)
41 | namedRdds.get[Int]("rdd1") should equal (Some(rdd))
42 | }
43 |
44 | it("destroy() should do nothing when RDD with given name doesn't exist") {
45 | namedRdds.update("rdd1", sc.parallelize(Seq(1, 2, 3)))
46 | namedRdds.get[Int]("rdd1") should not equal None
47 | namedRdds.destroy("rdd2")
48 | namedRdds.get[Int]("rdd1") should not equal None
49 | }
50 |
51 | it("destroy() should destroy an RDD that exists") {
52 | namedRdds.update("rdd1", sc.parallelize(Seq(1, 2, 3)))
53 | namedRdds.get[Int]("rdd1") should not equal None
54 | namedRdds.destroy("rdd1")
55 | namedRdds.get[Int]("rdd1") should equal (None)
56 | }
57 |
58 | it("getNames() should return names of all managed RDDs") {
59 | namedRdds.getNames() should equal (Iterable())
60 | namedRdds.update("rdd1", sc.parallelize(Seq(1, 2, 3)))
61 | namedRdds.update("rdd2", sc.parallelize(Seq(4, 5, 6)))
62 | namedRdds.getNames().toSeq.sorted should equal (Seq("rdd1", "rdd2"))
63 | namedRdds.destroy("rdd1")
64 | namedRdds.getNames().toSeq.sorted should equal (Seq("rdd2"))
65 | }
66 |
67 | it("getOrElseCreate() should call generator function if RDD does not exist") {
68 | var generatorCalled = false
69 | val rdd = namedRdds.getOrElseCreate("rdd1", {
70 | generatorCalled = true
71 | sc.parallelize(Seq(1, 2, 3))
72 | })
73 | generatorCalled should equal (true)
74 | }
75 |
76 | it("getOrElseCreate() should not call generator function, should return existing RDD if one exists") {
77 | var generatorCalled = false
78 | val rdd = sc.parallelize(Seq(1, 2, 3))
79 | namedRdds.update("rdd1", rdd)
80 | val rdd2 = namedRdds.getOrElseCreate("rdd1", {
81 | generatorCalled = true
82 | sc.parallelize(Seq(4, 5, 6))
83 | })
84 | generatorCalled should equal (false)
85 | rdd2 should equal (rdd)
86 | }
87 |
88 | it("update() should replace existing RDD") {
89 | val rdd1 = sc.parallelize(Seq(1, 2, 3))
90 | val rdd2 = sc.parallelize(Seq(4, 5, 6))
91 | namedRdds.getOrElseCreate("rdd", rdd1) should equal (rdd1)
92 | namedRdds.update("rdd", rdd2)
93 | namedRdds.get[Int]("rdd") should equal (Some(rdd2))
94 | }
95 |
96 | it("should include underlying exception when error occurs") {
97 | def errorFunc = {
98 | throw new IllegalArgumentException("boo!")
99 | sc.parallelize(Seq(1, 2))
100 | }
101 | val err = intercept[RuntimeException] { namedRdds.getOrElseCreate("rdd", errorFunc) }
102 | err.getCause.getClass should equal (classOf[IllegalArgumentException])
103 | }
104 | // TODO: Add tests for parallel getOrElseCreate() calls
105 | }
106 | }
107 |
--------------------------------------------------------------------------------
/job-server/test/spark.jobserver/SparkJobSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import org.scalatest.FunSpec
4 | import org.scalatest.matchers.ShouldMatchers
5 |
6 | class SparkJobSpec extends FunSpec with ShouldMatchers{
7 | val validSparkValidation = SparkJobValid
8 | val invalidSparkValidation = SparkJobInvalid("Sample message 1")
9 | val invalidSparkValidation2 = SparkJobInvalid("Sample message 2")
10 |
11 | describe("Sample tests for default validation && method") {
12 | it("should return valid") {
13 | validSparkValidation && (validSparkValidation) should equal (SparkJobValid)
14 | }
15 |
16 | it("should return invalid if one of them is invalid") {
17 | validSparkValidation && (invalidSparkValidation) should equal (SparkJobInvalid("Sample message 1"))
18 | }
19 |
20 | it ("should return invalid if both of them are invalid with the first message") {
21 | invalidSparkValidation2 && (invalidSparkValidation) should equal (SparkJobInvalid("Sample message 2"))
22 | }
23 | }
24 | }
25 |
--------------------------------------------------------------------------------
/job-server/test/spark.jobserver/SparkWebUiActorSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import akka.actor.{Props, ActorRef, ActorSystem}
4 | import akka.io.IO
5 | import akka.pattern.ask
6 | import akka.testkit.{TestKit, ImplicitSender}
7 | import com.typesafe.config.ConfigFactory
8 | import org.scalatest.{FunSpec, BeforeAndAfter, BeforeAndAfterAll}
9 | import org.scalatest.matchers.ShouldMatchers
10 | import spray.client.pipelining._
11 |
12 | import scala.concurrent.{Await, Future}
13 | import scala.concurrent.duration._
14 | import akka.util.Timeout
15 | import akka.actor._
16 | import spray.can.Http
17 | import spray.http._
18 | import HttpMethods._
19 | import SparkWebUiActor._
20 |
21 | import scala.util.{Failure, Success}
22 |
23 | object SparkWebUiActorSpec {
24 | val sparkWebUrl = "localhost"
25 | val sparkWebPort = 8098
26 | val config = ConfigFactory.parseString(s"""
27 | spark {
28 | master = "spark://badhost:7077,localhost:7077"
29 | webUrlPort = $sparkWebPort
30 | temp-contexts {
31 | num-cpu-cores = 4 # Number of cores to allocate. Required.
32 | memory-per-node = 512m # Executor memory per node, -Xmx style eg 512m, 1G, etc.
33 | }
34 | jobserver.job-result-cache-size = 100
35 | jobserver.job-result-cache-ttl-seconds = 60
36 | jobserver.context-creation-timeout = 5 s
37 | jobserver.context-factory = spark.jobserver.util.DefaultSparkContextFactory
38 | contexts {
39 | olap-demo {
40 | num-cpu-cores = 4
41 | memory-per-node = 512m
42 | }
43 | }
44 | context-settings {
45 | num-cpu-cores = 2
46 | memory-per-node = 512m
47 | }
48 | }
49 | akka.log-dead-letters = 0
50 | """)
51 |
52 | val system = ActorSystem("test", config)
53 | }
54 |
55 | // simple http service
56 | class SimpleHttpServer extends Actor with ActorLogging {
57 | implicit val timeout: Timeout = 1.second // for the actor 'asks'
58 | import context.dispatcher // ExecutionContext for the futures and scheduler
59 |
60 | def receive = {
61 | // when a new connection comes in we register ourselves as the connection handler
62 | case _: Http.Connected => sender ! Http.Register(self)
63 | case HttpRequest(GET, Uri.Path("/"), _, _, _) => {
64 | sender ! HttpResponse(entity =
65 | """blah blah
66 | |