configString
4 | WebApi->LocalContextSupervisor: GetContext(contextName)
5 | LocalContextSupervisor->WebApi: (JobManager, JobResultActor)
6 | WebApi->JobManager: StartJob(appName, clasPatch, userConfig, asyncEvents | syncEvents)
7 | JobManager->JobStatusActor: Subscribe(jobId, WebApi, asyncEvents | syncEvents)
8 | JobManager->JobResultActor: Subscribe(jobId, WebApi, asyncEvents | syncEvents)
9 | JobManager->JobFuture: future{}
10 | JobFuture->JobStatusActor: JobInit
11 | JobFuture->JobStatusActor: JobStarted
12 | opt if async job
13 | JobStatusActor->WebApi: JobStarted
14 | WebApi->user: 202 + jobId
15 | end
16 | note over JobFuture: SparkJob.runJob
17 | JobFuture->JobStatusActor: JobFinished(jobId, now)
18 | JobFuture->JobResultActor: JobResult(jobId, result)
19 | note over JobResultActor: cacheResult(jobId, result)
20 | opt if sync job
21 | JobResultActor->WebApi: JobResult(jobId, result)
22 | WebApi->user: 200 + JSON
23 | end
24 | JobFuture->JobStatusActor: Unsubscribe(jobId, WebApi)
25 | JobFuture->JobResultActor: Unsubscribe(jobId, WebApi)
26 |
--------------------------------------------------------------------------------
/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 |
--------------------------------------------------------------------------------
/akka-app/src/ooyala.common.akka/ActorStack.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka
2 |
3 | import akka.actor.Actor
4 |
5 | /**
6 | * A base trait for enabling stackable traits that enhance Actors.
7 | * Examples of stackable traits are included, and add logging, metrics, etc.
8 | *
9 | * == Actor classes ==
10 | * Actor classes that mix in this trait should define a wrappedReceive partial function
11 | * instead of the standard receive.
12 | *
13 | * Messages not handled by wrappedReceive will go, as usual, to unhandled().
14 | *
15 | * == Stacking traits ==
16 | * {{{
17 | * trait MyActorTrait extends ActorStack {
18 | * override def receive: Receive = {
19 | * case x =>
20 | * println("Before calling wrappedReceive... do something")
21 | * super.receive(x)
22 | * println("After calling wrappedReceive... do something else")
23 | * }
24 | * }
25 | * }}}
26 | */
27 | trait ActorStack extends Actor {
28 | /** Actor classes should implement this partialFunction for standard actor message handling */
29 | def wrappedReceive: Receive
30 |
31 | /** Stackable traits should override and call super.receive(x) for stacking functionality */
32 | def receive: Receive = {
33 | case x => if (wrappedReceive.isDefinedAt(x)) wrappedReceive(x) else unhandled(x)
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/akka-app/src/ooyala.common.akka/Slf4jLogging.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka
2 |
3 | import akka.actor.Actor
4 | import org.slf4j.LoggerFactory
5 |
6 | /**
7 | * Trait that adds Logback/SLF4J logging to actors. It adds the following members:
8 | *
9 | * * logger
10 | *
11 | * It also prints a message upon actor initialization.
12 | * Also, it fills the akkaSource MDC variable with the current actor's path, making for easier
13 | * log tracing of a single actor's messages.
14 | */
15 | trait Slf4jLogging extends ActorStack {
16 | val logger = LoggerFactory.getLogger(getClass)
17 | private[this] val myPath = self.path.toString
18 |
19 | withAkkaSourceLogging {
20 | logger.info("Starting actor " + getClass.getName)
21 | }
22 |
23 | override def receive: Receive = {
24 | case x =>
25 | withAkkaSourceLogging {
26 | super.receive(x)
27 | }
28 | }
29 |
30 | private def withAkkaSourceLogging(fn: => Unit) {
31 | // Because each actor receive invocation could happen in a different thread, and MDC is thread-based,
32 | // we kind of have to set the MDC anew for each receive invocation. :(
33 | try {
34 | org.slf4j.MDC.put("akkaSource", myPath)
35 | fn
36 | } finally {
37 | org.slf4j.MDC.remove("akkaSource")
38 | }
39 | }
40 | }
41 |
--------------------------------------------------------------------------------
/job-server-tests/src/spark.jobserver/WordCountExample.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import com.typesafe.config.{Config, ConfigFactory}
4 | import org.apache.spark._
5 | import org.apache.spark.SparkContext._
6 | import scala.util.Try
7 |
8 | /**
9 | * A super-simple Spark job example that implements the SparkJob trait and can be submitted to the job server.
10 | *
11 | * Set the config with the sentence to split or count:
12 | * input.string = "adsfasdf asdkf safksf a sdfa"
13 | *
14 | * validate() returns SparkJobInvalid if there is no input.string
15 | */
16 | object WordCountExample extends SparkJob {
17 | def main(args: Array[String]) {
18 | val sc = new SparkContext("local[4]", "WordCountExample")
19 | val config = ConfigFactory.parseString("")
20 | val results = runJob(sc, config)
21 | println("Result is " + results)
22 | }
23 |
24 | override def validate(sc: SparkContext, config: Config): SparkJobValidation = {
25 | Try(config.getString("input.string"))
26 | .map(x => SparkJobValid)
27 | .getOrElse(SparkJobInvalid("No input.string config param"))
28 | }
29 |
30 | override def runJob(sc: SparkContext, config: Config): Any = {
31 | val dd = sc.parallelize(config.getString("input.string").split(" ").toSeq)
32 | dd.map((_, 1)).reduceByKey(_ + _).collect().toMap
33 | }
34 | }
35 |
--------------------------------------------------------------------------------
/akka-app/src/ooyala.common.akka/actor/Reaper.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka.actor
2 |
3 | import akka.actor.{ActorRef, Terminated}
4 | import scala.collection.mutable.ArrayBuffer
5 | import ooyala.common.akka.InstrumentedActor
6 |
7 | // Taken from http://letitcrash.com/post/30165507578/shutdown-patterns-in-akka-2
8 |
9 | object Reaper {
10 | // Used by others to register an Actor for watching
11 | case class WatchMe(ref: ActorRef)
12 | case object Reaped
13 | }
14 |
15 | abstract class Reaper extends InstrumentedActor {
16 | import Reaper._
17 |
18 | // Keep track of what we're watching
19 | val watched = ArrayBuffer.empty[ActorRef]
20 |
21 | def allSoulsReaped(): Unit
22 |
23 | // Watch and check for termination
24 | override def wrappedReceive: Receive = {
25 | case Reaped =>
26 | watched.isEmpty
27 |
28 | case WatchMe(ref) =>
29 | logger.info("Watching actor {}", ref)
30 | context.watch(ref)
31 | watched += ref
32 |
33 | case Terminated(ref) =>
34 | logger.info("Actor {} terminated", ref)
35 | watched -= ref
36 | if (watched.isEmpty) allSoulsReaped()
37 | }
38 | }
39 |
40 | class ProductionReaper extends Reaper {
41 | def allSoulsReaped() {
42 | logger.warn("Shutting down actor system because all actors have terminated")
43 | context.system.shutdown()
44 | }
45 | }
46 |
47 |
--------------------------------------------------------------------------------
/job-server/config/logback-server.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 |
6 |
7 |
8 |
9 |
10 |
11 |
12 |
13 |
14 | ${LOG_DIR}/spark-job-server.log
15 |
16 |
17 |
18 | ${LOG_DIR}/spark-job-server.%d{yyyy-MM-dd}.log
19 |
20 |
21 | 30
22 |
23 |
24 |
25 | [%date{ISO8601}] %-5level %logger{20} [%X{jobId}] [%X{akkaSource}] - %msg%n
26 |
27 |
28 |
29 |
30 |
31 |
--------------------------------------------------------------------------------
/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 |
--------------------------------------------------------------------------------
/akka-app/test/ooyala.common.akka/ActorStackSpec.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka
2 |
3 | import org.scalatest.matchers.ShouldMatchers
4 | import org.scalatest.FunSpec
5 | import akka.testkit.TestActorRef
6 |
7 | import akka.actor.{Actor, ActorSystem}
8 |
9 |
10 | class DummyActor extends ActorStack {
11 | var str = ""
12 | def wrappedReceive = {
13 | case s: String => str += s
14 | }
15 |
16 | override def unhandled(x: Any) { str = "unhandled" }
17 | }
18 |
19 | trait AddPrefix extends ActorStack {
20 | override def receive: Receive = {
21 | case x: String =>
22 | super.receive("pre " + x.asInstanceOf[String])
23 | }
24 | }
25 |
26 | class ActorStackSpec extends FunSpec with ShouldMatchers {
27 | implicit val system = ActorSystem("test")
28 |
29 | describe("stacking traits") {
30 | it("should be able to stack traits and receive messages") {
31 | val actorRef = TestActorRef(new DummyActor with AddPrefix)
32 | val actor = actorRef.underlyingActor
33 |
34 | actorRef ! "me"
35 | actor.str should equal ("pre me")
36 | }
37 |
38 | it("should pass messages not handled in wrappedReceive to unhandled function") {
39 | val actorRef = TestActorRef[DummyActor]
40 | val actor = actorRef.underlyingActor
41 |
42 | actorRef ! List(1, 2)
43 | actor.str should equal ("unhandled")
44 | }
45 | }
46 | }
--------------------------------------------------------------------------------
/job-server/test/spark.jobserver/util/SparkJobUtilsSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.util
2 |
3 | import com.typesafe.config.ConfigFactory
4 | import org.apache.spark.SparkConf
5 | import org.scalatest.FunSpec
6 | import org.scalatest.matchers.ShouldMatchers
7 |
8 | class SparkJobUtilsSpec extends FunSpec with ShouldMatchers {
9 | import collection.JavaConverters._
10 |
11 | val config = ConfigFactory.parseMap(Map(
12 | "spark.home" -> "/etc/spark",
13 | "spark.master" -> "local[4]"
14 | ).asJava)
15 | val contextName = "demo"
16 |
17 | def getSparkConf(configMap: Map[String, Any]): SparkConf =
18 | SparkJobUtils.configToSparkConf(config, ConfigFactory.parseMap(configMap.asJava), contextName)
19 |
20 | describe("SparkJobUtils.configToSparkConf") {
21 | it("should translate num-cpu-cores and memory-per-node properly") {
22 | val sparkConf = getSparkConf(Map("num-cpu-cores" -> 4, "memory-per-node" -> "512m"))
23 | sparkConf.get("spark.master") should equal ("local[4]")
24 | sparkConf.get("spark.cores.max") should equal ("4")
25 | sparkConf.get("spark.executor.memory") should equal ("512m")
26 | sparkConf.get("spark.home") should equal ("/etc/spark")
27 | }
28 |
29 | it("should add other arbitrary settings") {
30 | val sparkConf = getSparkConf(Map("spark.cleaner.ttl" -> 86400))
31 | sparkConf.getInt("spark.cleaner.ttl", 0) should equal (86400)
32 | }
33 | }
34 | }
--------------------------------------------------------------------------------
/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/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/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 |
--------------------------------------------------------------------------------
/akka-app/test/ooyala.common.akka/metrics/JvmMetricsWrapperSpec.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka.metrics
2 |
3 | import com.codahale.metrics.{Metric, MetricFilter}
4 | import org.scalatest.{BeforeAndAfter, FunSpec}
5 | import org.scalatest.matchers.ShouldMatchers
6 |
7 | class JvmMetricsWrapperSpec extends FunSpec with ShouldMatchers with BeforeAndAfter {
8 | private val metricsRegistry = MetricsWrapper.getRegistry
9 | private val metricNamePrefix = "spark.jobserver.jvm"
10 |
11 | def removeJvmMetrics = {
12 | metricsRegistry.removeMatching(new MetricFilter {
13 | override def matches(name: String, metric: Metric): Boolean = {
14 | return name.startsWith(metricNamePrefix)
15 | }
16 | })
17 | }
18 |
19 | before {
20 | removeJvmMetrics
21 | JvmMetricsWrapper.registerJvmMetrics(metricsRegistry)
22 | }
23 |
24 | after {
25 | removeJvmMetrics
26 | }
27 |
28 | describe("JvmMetricsWrapper") {
29 | it("should have valid metrics") {
30 | val gauges = metricsRegistry.getGauges
31 |
32 | gauges.get(metricNamePrefix + ".thread.count").getValue should not equal(0)
33 | gauges.get(metricNamePrefix + ".thread.daemon.count").getValue should not equal(0)
34 |
35 | gauges.get(metricNamePrefix + ".heap.committed").getValue should not equal(0)
36 | gauges.get(metricNamePrefix + ".heap.used").getValue should not equal(0)
37 |
38 | gauges.get(metricNamePrefix + ".non-heap.committed").getValue should not equal(0)
39 | gauges.get(metricNamePrefix + ".non-heap.used").getValue should not equal(0)
40 | }
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/doc/start-job-msg-flow-AdHocJobManagerActor.md:
--------------------------------------------------------------------------------
1 | title POST /jobs start new job workflow with AdHocJobManagerActor
2 |
3 | user->WebApi: POST /jobs
4 |
5 | WebApi->LocalContextSupervisor: GetAdHocContext
6 | LocalContextSupervisor->WebApi: (AdHocJobManager, JobResultActor)
7 |
8 | WebApi->AdHocJobManagerActor: StartJob(event for JobStatusActor)
9 |
10 | note over AdHocJobManagerActor: validate appName, className
11 |
12 | opt if Job validation fails
13 | AdHocJobManagerActor->WebApi: ERROR
14 | WebApi->user: 400
15 | end
16 |
17 | AdHocJobManagerActor->JobStatusActor: Subscribe(jobId, WebApi, event)
18 | AdHocJobManagerActor->JobResultActor: Subscribe(jobId, WebApi, JobResult)
19 | AdHocJobManagerActor->JobFuture: CreateJob
20 |
21 | JobFuture->JobStatusActor: JobInit(info)
22 |
23 | note over JobFuture: SparkJob.validate()
24 |
25 | opt if validation fails
26 | JobFuture->JobStatusActor: ValidationFailed
27 | JobStatusActor->WebApi: ValidationFailed
28 | WebApi->user: 400
29 | end
30 |
31 | JobFuture->JobStatusActor: JobStarted(jobId)
32 |
33 | opt if async job
34 | JobStatusActor->WebApi: JobStarted(jobId)
35 | WebApi->user: 202
36 | end
37 |
38 | note over JobFuture: SparkJob.runJob()
39 |
40 | JobFuture->JobStatusActor: JobFinish(jobId)
41 | JobFuture->JobStatusActor: Unsubscribe(jobId, WebApi)
42 |
43 | JobFuture->JobResultActor: JobResult(jobId, result)
44 |
45 | JobFuture->AdHocJobManagerActor: JobFinish(jobId)
46 |
47 | note over JobFuture: Terminate
48 |
49 | note over JobResultActor: cacheResult(jobId, result)
50 |
51 | opt if sync job
52 | JobResultActor->WebApi: JobResult(jobId, result)
53 | WebApi->user: result
54 | end
55 |
--------------------------------------------------------------------------------
/akka-app/src/ooyala.common.akka/metrics/JvmMetricsWrapper.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka.metrics
2 |
3 | import com.codahale.metrics.MetricRegistry
4 | import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet, ThreadStatesGaugeSet}
5 | import scala.collection.JavaConverters._
6 |
7 | /**
8 | * JVM metrics wrapper
9 | */
10 | object JvmMetricsWrapper {
11 | // JVM metrics Sets
12 | private val threadMetricSet = new ThreadStatesGaugeSet
13 | private val memoryMetricSet = new MemoryUsageGaugeSet
14 | private val gcMetricSet = new GarbageCollectorMetricSet
15 | private val metricNamePrefix = "spark.jobserver.jvm"
16 |
17 | /**
18 | * Registers JVM metrics to registry
19 | *
20 | * @param registry the registry to register JVM metrics
21 | */
22 | def registerJvmMetrics(registry: MetricRegistry) = {
23 | registry.register(metricNamePrefix + ".thread.count", threadMetricSet.getMetrics.get("count"))
24 | registry.register(metricNamePrefix + ".thread.daemon.count", threadMetricSet.getMetrics
25 | .get("daemon.count"))
26 | registry.register(metricNamePrefix + ".heap.committed", memoryMetricSet.getMetrics
27 | .get("heap.committed"))
28 | registry.register(metricNamePrefix + ".heap.used", memoryMetricSet.getMetrics.get("heap.used"))
29 | registry.register(metricNamePrefix + ".non-heap.committed", memoryMetricSet.getMetrics
30 | .get("non-heap.committed"))
31 | registry.register(metricNamePrefix + ".non-heap.used", memoryMetricSet.getMetrics.get("non-heap.used"))
32 |
33 | // Registers gc metrics
34 | gcMetricSet.getMetrics.asScala.foreach {
35 | case (name, metric) => {
36 | registry.register(metricNamePrefix + ".gc." + name, metric)
37 | }
38 | }
39 | }
40 | }
41 |
--------------------------------------------------------------------------------
/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-api/src/spark.jobserver/SparkJob.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import com.typesafe.config.Config
4 | import org.apache.spark.SparkContext
5 |
6 | sealed trait SparkJobValidation {
7 | // NOTE(harish): We tried using lazy eval here by passing in a function
8 | // instead, which worked fine with tests but when run with the job-server
9 | // it would just hang and timeout. This is something worth investigating
10 | def &&(sparkValidation: SparkJobValidation): SparkJobValidation = this match {
11 | case SparkJobValid => sparkValidation
12 | case x => x
13 | }
14 | }
15 | case object SparkJobValid extends SparkJobValidation
16 | case class SparkJobInvalid(reason: String) extends SparkJobValidation
17 |
18 | /**
19 | * This trait is the main API for Spark jobs submitted to the Job Server.
20 | */
21 | trait SparkJob {
22 | /**
23 | * This is the entry point for a Spark Job Server to execute Spark jobs.
24 | * This function should create or reuse RDDs and return the result at the end, which the
25 | * Job Server will cache or display.
26 | * @param sc a SparkContext for the job. May be reused across jobs.
27 | * @param jobConfig the Typesafe Config object passed into the job request
28 | * @return the job result
29 | */
30 | def runJob(sc: SparkContext, jobConfig: Config): Any
31 |
32 | /**
33 | * This method is called by the job server to allow jobs to validate their input and reject
34 | * invalid job requests. If SparkJobInvalid is returned, then the job server returns 400
35 | * to the user.
36 | * NOTE: this method should return very quickly. If it responds slowly then the job server may time out
37 | * trying to start this job.
38 | * @return either SparkJobValid or SparkJobInvalid
39 | */
40 | def validate(sc: SparkContext, config: Config): SparkJobValidation
41 | }
42 |
--------------------------------------------------------------------------------
/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 | }
--------------------------------------------------------------------------------
/akka-app/test/ooyala.common.akka/actor/ReaperSpec.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka.actor
2 |
3 |
4 | import akka.actor.{ActorSystem, Props, ActorRef}
5 | import akka.testkit.{TestKit, ImplicitSender, TestProbe}
6 | import org.scalatest.{FunSpec, WordSpec, BeforeAndAfterAll}
7 | import org.scalatest.matchers.MustMatchers
8 |
9 | // Our test reaper. Sends the snooper a message when all
10 | // the souls have been reaped
11 | class TestReaper(snooper: ActorRef) extends Reaper {
12 | def allSoulsReaped(): Unit = snooper ! "Dead"
13 | }
14 |
15 | class ReaperSpec extends TestKit(ActorSystem("ReaperSpec")) with ImplicitSender
16 | with FunSpec
17 | with BeforeAndAfterAll
18 | with MustMatchers {
19 |
20 | import Reaper._
21 | import scala.concurrent.duration._
22 |
23 | override def afterAll() {
24 | system.shutdown()
25 | }
26 |
27 | describe("Reaper") {
28 | it("should not call allSoulsReaped if not all actors are done") {
29 | val a = TestProbe()
30 | val d = TestProbe()
31 |
32 | // Build our reaper
33 | val reaper = system.actorOf(Props(classOf[TestReaper], testActor))
34 |
35 | // Watch a couple
36 | reaper ! WatchMe(a.ref)
37 | reaper ! WatchMe(d.ref)
38 |
39 | // Stop one of them
40 | system.stop(a.ref)
41 |
42 | expectNoMsg(500 millis)
43 | }
44 |
45 | it("should detect that all actors can be reaped") {
46 | // Set up some dummy Actors
47 | val a = TestProbe()
48 | val d = TestProbe()
49 |
50 | // Build our reaper
51 | val reaper = system.actorOf(Props(classOf[TestReaper], testActor))
52 |
53 | // Watch a couple
54 | reaper ! WatchMe(a.ref)
55 | reaper ! WatchMe(d.ref)
56 |
57 | // Stop them
58 | system.stop(a.ref)
59 | system.stop(d.ref)
60 |
61 | // Make sure we've been called
62 | expectMsg("Dead")
63 | }
64 | }
65 | }
66 |
--------------------------------------------------------------------------------
/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/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/config/local.conf.template:
--------------------------------------------------------------------------------
1 | # Template for a Spark Job Server configuration file
2 | # When deployed these settings are loaded when job server starts
3 | #
4 | # Spark Cluster / Job Server configuration
5 | spark {
6 | # spark.master will be passed to each job's JobContext
7 | master = "local[4]"
8 | # master = "mesos://vm28-hulk-pub:5050"
9 |
10 | # Default # of CPUs for jobs to use for Spark standalone cluster
11 | job-number-cpus = 4
12 |
13 | jobserver {
14 | port = 8090
15 | jar-store-rootdir = /tmp/jobserver/jars
16 |
17 | jobdao = spark.jobserver.io.JobFileDAO
18 |
19 | filedao {
20 | rootdir = /tmp/spark-job-server/filedao/data
21 | }
22 | }
23 |
24 | # predefined Spark contexts
25 | # contexts {
26 | # my-low-latency-context {
27 | # num-cpu-cores = 1 # Number of cores to allocate. Required.
28 | # memory-per-node = 512m # Executor memory per node, -Xmx style eg 512m, 1G, etc.
29 | # }
30 | # # define additional contexts here
31 | # }
32 |
33 | # universal context configuration. These settings can be overridden, see README.md
34 | context-settings {
35 | num-cpu-cores = 2 # Number of cores to allocate. Required.
36 | memory-per-node = 512m # Executor memory per node, -Xmx style eg 512m, #1G, etc.
37 |
38 | # in case spark distribution should be accessed from HDFS (as opposed to being installed on every mesos slave)
39 | # spark.executor.uri = "hdfs://namenode:8020/apps/spark/spark.tgz"
40 |
41 | # uris of jars to be loaded into the classpath for this context
42 | # dependent-jar-uris = ["file:///some/path/present/in/each/mesos/slave/somepackage.jar"]
43 | }
44 |
45 | # This needs to match SPARK_HOME for cluster SparkContexts to be created successfully
46 | # home = "/home/spark/spark"
47 | }
48 |
49 | # Note that you can use this file to define settings not only for job server,
50 | # but for your Spark jobs as well. Spark job configuration merges with this configuration file as defaults.
51 |
--------------------------------------------------------------------------------
/doc/actors.md:
--------------------------------------------------------------------------------
1 | # Introduction
2 |
3 | There are two separate ActorSystems or clusters of actors in the job server architecture.
4 |
5 | * JobServer system - this contains the actual REST API and manages all of the job context systems.
6 | * JobContext system - one system per JobContext.
7 |
8 | Each JobContext could potentially run in its own process for isolation purposes, and multiple JobServers may connect to the same JobContext for HA.
9 |
10 | # JobServer ActorSystem
11 |
12 | Here are all the actors for the job server.
13 |
14 | ## WebApi
15 |
16 | This is not really an actor but contains the web routes.
17 |
18 | ## ContextSupervisor
19 |
20 | - Creates and stops JobContext actorsystems
21 | - Sends jobs on to job contexts
22 | - is a singleton
23 |
24 | # JobContext ActorSystem
25 |
26 | ## JobManager
27 |
28 | This was the "ContextManager" actor.
29 |
30 | - one per context
31 | - Starts JobActors for every job in the context
32 | - returns an error if there are no more threads for jobs or capacity is full
33 | - Starts and supervises the JobStatus and JobResult actors
34 |
35 | ## AdHocJobManager
36 |
37 | A special JobManager for running ad-hoc jobs, which require temporary per-job JobContexts.
38 |
39 | - When the job terminates, the JobManager cleans up the SparkContext.
40 |
41 | ## JobStatusActor
42 |
43 | - one per JobManager
44 | - Collects and persists job status and progress updates (including exceptions) from every job in JobManager
45 | - JDBC updates
46 | - log file
47 | - WebSocket?
48 | - Handles subscriptions from external actors for listening to status updates for specific jobID's
49 | - Watches the JobActors, removing subscriptions once the actor terminates
50 |
51 | ## JobResultActor
52 |
53 | - one per JobManager
54 | - Collects job results
55 | - For now, do not persist it, just keep in memory
56 | - handles requests from external actors for the job results
57 |
58 | ## JobActor
59 |
60 | - many per JobManager
61 | - Invokes SparkJob.validate(), SparkJob.runJob(), etc.
62 | - sends status updates back to JobStatus, JobResult actors
63 |
--------------------------------------------------------------------------------
/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/main/resources/application.conf:
--------------------------------------------------------------------------------
1 | # Settings for safe local mode development
2 | spark {
3 | master = "local[4]"
4 | # spark web UI port
5 | webUrlPort = 8080
6 |
7 | jobserver {
8 | port = 8090
9 |
10 | # Number of job results to keep per JobResultActor/context
11 | job-result-cache-size = 5000
12 |
13 | # TTL for cache antries to avoid high memory usage
14 | job-result-cache-ttl-seconds = 600
15 |
16 | jobdao = spark.jobserver.io.JobFileDAO
17 |
18 | filedao {
19 | rootdir = /tmp/spark-jobserver/filedao/data
20 | }
21 |
22 | # Time out for job server to wait while creating contexts
23 | context-creation-timeout = 15 s
24 |
25 | # A zero-arg class implementing spark.jobserver.util.SparkContextFactory
26 | context-factory = spark.jobserver.util.DefaultSparkContextFactory
27 | }
28 |
29 | # predefined Spark contexts
30 | # Below is an example, but do not uncomment it. Everything defined here is carried over to
31 | # deploy-time configs, so they will be created in all environments. :(
32 | contexts {
33 | # abc-demo {
34 | # num-cpu-cores = 4 # Number of cores to allocate. Required.
35 | # memory-per-node = 1024m # Executor memory per node, -Xmx style eg 512m, 1G, etc.
36 | # }
37 | # define additional contexts here
38 | }
39 |
40 | # Default settings for ad hoc as well as manually created contexts
41 | # You can add any Spark config params here, for example, spark.mesos.coarse = true
42 | context-settings {
43 | num-cpu-cores = 4 # Number of cores to allocate. Required.
44 | memory-per-node = 512m # Executor memory per node, -Xmx style eg 512m, 1G, etc.
45 | # max-jobs-per-context = 4 # Max # of jobs to run at the same time
46 | }
47 | }
48 |
49 | akka {
50 | # Use SLF4J/logback for deployed environment logging
51 | loggers = ["akka.event.slf4j.Slf4jLogger"]
52 | }
53 |
54 | # check the reference.conf in spray-can/src/main/resources for all defined settings
55 | spray.can.server {
56 | # uncomment the next line for making this an HTTPS example
57 | # ssl-encryption = on
58 | idle-timeout = 20 s
59 | request-timeout = 15 s
60 | pipelining-limit = 2 # for maximum performance (prevents StopReading / ResumeReading messages to the IOBridge)
61 | # Needed for HTTP/1.0 requests with missing Host headers
62 | default-host-header = "spray.io:8765"
63 | parsing {
64 | max-content-length = 200m
65 | }
66 | }
67 |
--------------------------------------------------------------------------------
/job-server/src/main/resources/html/js/spark-jobserver-ui.js:
--------------------------------------------------------------------------------
1 | function getJobs() {
2 | $.getJSON(
3 | '/jobs',
4 | '',
5 | function(jobs) {
6 | $('#failedJobsTable tbody').empty();
7 | $('#runningJobsTable tbody').empty();
8 | $('#completedJobsTable tbody').empty();
9 |
10 | $.each(jobs, function(key, job) {
11 | var items = [];
12 | items.push("");
13 | items.push("" + job.jobId + " ");
14 | items.push("" + job.classPath + " ");
15 | items.push("" + job.context + " ");
16 | items.push("" + job.startTime + " ");
17 | items.push("" + job.duration + " ");
18 | items.push(" ");
19 |
20 | if(job.status == 'ERROR') {
21 | $('#failedJobsTable > tbody:last').append(items.join(""));
22 | } else if(job.status == 'RUNNING') {
23 | $('#runningJobsTable > tbody:last').append(items.join(""));
24 | } else {
25 | $('#completedJobsTable > tbody:last').append(items.join(""));
26 | }
27 | });
28 | });
29 | }
30 |
31 | function getContexts() {
32 | $.getJSON(
33 | '/contexts',
34 | '',
35 | function(contexts) {
36 | $('#contextsTable tbody').empty();
37 |
38 | $.each(contexts, function(key, contextName) {
39 | var items = [];
40 | items.push("" + contextName + " ");
41 | $('#contextsTable > tbody:last').append(items.join(""));
42 | });
43 | });
44 | }
45 |
46 | function getJars() {
47 | $.getJSON(
48 | '/jars',
49 | '',
50 | function(jars) {
51 | $('#jarsTable tbody').empty();
52 |
53 | $.each(jars, function(jarName, deploymentTime) {
54 | var items = [];
55 | items.push("");
56 | items.push("" + jarName + " ");
57 | items.push("" + deploymentTime + " ");
58 | items.push(" ");
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/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 |
--------------------------------------------------------------------------------
/bin/server_start.sh:
--------------------------------------------------------------------------------
1 | #!/bin/bash
2 | # Script to start the job server
3 | set -e
4 |
5 | get_abs_script_path() {
6 | pushd . >/dev/null
7 | cd $(dirname $0)
8 | appdir=$(pwd)
9 | popd >/dev/null
10 | }
11 |
12 | get_abs_script_path
13 |
14 | GC_OPTS="-XX:+UseConcMarkSweepGC
15 | -verbose:gc -XX:+PrintGCTimeStamps -Xloggc:$appdir/gc.out
16 | -XX:MaxPermSize=512m
17 | -XX:+CMSClassUnloadingEnabled "
18 |
19 | JAVA_OPTS="-Xmx5g -XX:MaxDirectMemorySize=512M
20 | -XX:+HeapDumpOnOutOfMemoryError -Djava.net.preferIPv4Stack=true
21 | -Dcom.sun.management.jmxremote.port=9999
22 | -Dcom.sun.management.jmxremote.authenticate=false
23 | -Dcom.sun.management.jmxremote.ssl=false"
24 |
25 | MAIN="spark.jobserver.JobServer"
26 |
27 | conffile=$(ls -1 $appdir/*.conf | head -1)
28 | if [ -z "$conffile" ]; then
29 | echo "No configuration file found"
30 | exit 1
31 | fi
32 |
33 | if [ -f "$appdir/settings.sh" ]; then
34 | . $appdir/settings.sh
35 | else
36 | echo "Missing $appdir/settings.sh, exiting"
37 | exit 1
38 | fi
39 |
40 | if [ -z "$SPARK_HOME" ]; then
41 | echo "Please set SPARK_HOME or put it in $appdir/settings.sh first"
42 | exit 1
43 | fi
44 |
45 | if [ -z "$SPARK_CONF_HOME" ]; then
46 | SPARK_CONF_HOME=$SPARK_HOME/conf
47 | fi
48 |
49 | # Pull in other env vars in spark config, such as MESOS_NATIVE_LIBRARY
50 | . $SPARK_CONF_HOME/spark-env.sh
51 |
52 | if [ -f "$PIDFILE" ] && kill -0 $(cat "$PIDFILE"); then
53 | echo 'Job server is already running'
54 | exit 1
55 | fi
56 |
57 | if [ -z "$LOG_DIR" ]; then
58 | echo "LOG_DIR empty; logging will go to /tmp/job-server"
59 | LOG_DIR=/tmp/job-server
60 | fi
61 | mkdir -p $LOG_DIR
62 |
63 | LOGGING_OPTS="-Dlog4j.configuration=log4j-server.properties
64 | -DLOG_DIR=$LOG_DIR"
65 |
66 | # For Mesos
67 | CONFIG_OVERRIDES="-Dspark.executor.uri=$SPARK_EXECUTOR_URI "
68 | # For Mesos/Marathon, use the passed-in port
69 | if [ "$PORT" != "" ]; then
70 | CONFIG_OVERRIDES+="-Dspark.jobserver.port=$PORT "
71 | fi
72 |
73 | # This needs to be exported for standalone mode so drivers can connect to the Spark cluster
74 | export SPARK_HOME
75 |
76 | # job server jar needs to appear first so its deps take higher priority
77 | # need to explicitly include app dir in classpath so logging configs can be found
78 | CLASSPATH="$appdir:$appdir/spark-job-server.jar:$($SPARK_HOME/bin/compute-classpath.sh)"
79 |
80 | exec java -cp $CLASSPATH $GC_OPTS $JAVA_OPTS $LOGGING_OPTS $CONFIG_OVERRIDES $MAIN $conffile 2>&1 &
81 | echo $! > $PIDFILE
82 |
--------------------------------------------------------------------------------
/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-tests/src/spark.jobserver/SparkTestJobs.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import com.typesafe.config.Config
4 | import org.apache.spark._
5 | import org.apache.spark.rdd.RDD
6 |
7 |
8 | trait SparkTestJob extends SparkJob {
9 | def validate(sc: SparkContext, config: Config): SparkJobValidation = SparkJobValid
10 | }
11 |
12 | class MyErrorJob extends SparkTestJob {
13 | def runJob(sc: SparkContext, config: Config): Any = {
14 | throw new IllegalArgumentException("Foobar")
15 | }
16 | }
17 |
18 | class ConfigCheckerJob extends SparkTestJob {
19 | import scala.collection.JavaConverters._
20 |
21 | def runJob(sc: SparkContext, config: Config): Any = {
22 | config.root().keySet().asScala.toSeq
23 | }
24 | }
25 |
26 | // A simple test job that sleeps for a configurable time. Used to test the max-running-jobs feature.
27 | class SleepJob extends SparkTestJob {
28 | def runJob(sc: SparkContext, config: Config): Any = {
29 | val sleepTimeMillis: Long = config.getLong("sleep.time.millis")
30 | Thread.sleep(sleepTimeMillis)
31 | sleepTimeMillis
32 | }
33 | }
34 |
35 | class CacheSomethingJob extends SparkTestJob {
36 | def runJob(sc: SparkContext, config: Config): Any = {
37 | val dd = sc.parallelize(Seq(2, 4, 9, 16, 25, 36, 55, 66))
38 | .map(_ * 2)
39 | dd.setName("numbers")
40 | dd.cache()
41 | dd.collect.toSeq.sum
42 | }
43 | }
44 |
45 | class AccessCacheJob extends SparkTestJob {
46 | def runJob(sc: SparkContext, config: Config): Any = {
47 | val rdd = sc.getPersistentRDDs.values.head.asInstanceOf[RDD[Int]]
48 | rdd.collect.toSeq.sum
49 | }
50 | }
51 |
52 | class CacheRddByNameJob extends SparkTestJob with NamedRddSupport {
53 | def runJob(sc: SparkContext, config: Config): Any = {
54 | import scala.concurrent.duration._
55 | implicit val timeout = akka.util.Timeout(100 millis)
56 |
57 | val rdd = namedRdds.getOrElseCreate(getClass.getSimpleName, {
58 | // anonymous generator function
59 | sc.parallelize(Seq(1, 2, 3, 4, 5))
60 | })
61 |
62 | // RDD should already be in cache the second time
63 | val rdd2 = namedRdds.get[Int](getClass.getSimpleName)
64 | assert(rdd2 == Some(rdd), "Error: " + rdd2 + " != " + Some(rdd))
65 | rdd.map { x => x * x }.toArray().sum
66 | }
67 | }
68 |
69 | case class Animal(name: String)
70 |
71 | class ZookeeperJob extends SparkTestJob {
72 | def runJob(sc: SparkContext, config: Config): Any = {
73 | val dd = sc.parallelize(Seq(Animal("dog"), Animal("cat"), Animal("horse")))
74 | dd.filter(animal => animal.name.startsWith("ho")).collect()
75 | }
76 | }
77 |
78 | object SimpleObjectJob extends SparkTestJob {
79 | def runJob(sc: SparkContext, config: Config): Any = {
80 | val rdd = sc.parallelize(Seq(1, 2, 3))
81 | rdd.toArray().sum
82 | }
83 | }
84 |
--------------------------------------------------------------------------------
/job-server-tests/src/spark.jobserver/LongPiJob.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver
2 |
3 | import com.typesafe.config.{Config, ConfigFactory}
4 | import org.apache.spark._
5 | import scala.util.Try
6 | import java.util.{Random, Date}
7 |
8 | /**
9 | * A long job for stress tests purpose.
10 | * Iterative and randomized algorithm to compute Pi.
11 | * Imagine a square centered at (1,1) of length 2 units.
12 | * It tightly encloses a circle centered also at (1,1) of radius 1 unit.
13 | * Randomly throw darts to them.
14 | * We can use the ratio of darts inside the square and circle to approximate the Pi.
15 | *
16 | * stress.test.longpijob.duration controls how long it run in seconds.
17 | * Longer duration increases precision.
18 | *
19 | */
20 | object LongPiJob extends SparkJob {
21 | private val rand = new Random(now)
22 |
23 | def main(args: Array[String]) {
24 | val sc = new SparkContext("local[4]", "LongPiJob")
25 | val config = ConfigFactory.parseString("")
26 | val results = runJob(sc, config)
27 | println("Result is " + results)
28 | }
29 |
30 | override def validate(sc: SparkContext, config: Config): SparkJobValidation = {
31 | SparkJobValid
32 | }
33 |
34 | override def runJob(sc: SparkContext, config: Config): Any = {
35 | val duration = Try(config.getInt("stress.test.longpijob.duration")).getOrElse(5)
36 | var hit:Long = 0
37 | var total:Long = 0
38 | val start = now
39 | while(stillHaveTime(start, duration)) {
40 | val counts = estimatePi(sc)
41 | hit = hit + counts._1
42 | total = total + counts._2
43 | }
44 |
45 | (4.0 * hit) / total
46 | }
47 |
48 | /**
49 | *
50 | * @param sc
51 | * @return (hit, total) where hit is the count hit inside circle and total is the total darts
52 | */
53 | private def estimatePi(sc: SparkContext): Tuple2[Int, Int] = {
54 | val data = Array.iterate(0, 1000)(x => x + 1)
55 |
56 | val dd = sc.parallelize(data)
57 | dd.map { x =>
58 | // The first value is the count of hitting inside the circle. The second is the total.
59 | if (throwADart()) (1, 1) else (0, 1)
60 | }.reduce { (x, y) => (x._1 + y._1, x._2 + y._2) }
61 | }
62 |
63 | /**
64 | * Throw a dart.
65 | *
66 | * @return true if the dart hits inside the circle.
67 | */
68 | private def throwADart(): Boolean = {
69 | val x = rand.nextDouble() * 2
70 | val y = rand.nextDouble() * 2
71 | // square of distance to center
72 | val dist = math.pow(x - 1, 2) + math.pow(y - 1, 2)
73 | // square root wouldn't affect the math.
74 | // if dist > 1, then hit outside the circle, else hit inside the circle
75 | dist <= 1
76 | }
77 |
78 | private def now(): Long = (new Date()).getTime
79 |
80 | private val OneSec = 1000 // in milliseconds
81 | private def stillHaveTime(startTime: Long, duration: Int): Boolean = (now - startTime) < duration * OneSec
82 | }
83 |
--------------------------------------------------------------------------------
/project/Dependencies.scala:
--------------------------------------------------------------------------------
1 | import sbt._
2 | import Keys._
3 |
4 | object Dependencies {
5 | val excludeCglib = ExclusionRule(organization = "org.sonatype.sisu.inject")
6 | val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson")
7 | val excludeNetty = ExclusionRule(organization = "org.jboss.netty")
8 | val excludeAsm = ExclusionRule(organization = "asm")
9 |
10 | lazy val typeSafeConfigDeps = "com.typesafe" % "config" % "1.0.0"
11 | lazy val yammerDeps = "com.yammer.metrics" % "metrics-core" % "2.2.0"
12 |
13 | lazy val yodaDeps = Seq(
14 | "org.joda" % "joda-convert" % "1.2",
15 | "joda-time" % "joda-time" % "2.1"
16 | )
17 |
18 | lazy val akkaDeps = Seq(
19 | // Akka is provided because Spark already includes it, and Spark's version is shaded so it's not safe
20 | // to use this one
21 | "com.typesafe.akka" %% "akka-slf4j" % "2.2.4" % "provided",
22 | "io.spray" %% "spray-json" % "1.2.5",
23 | // upgrade version from 1.2.0 to 1.2.1 to solve the logging noise issue
24 | // details here: https://groups.google.com/forum/#!msg/spray-user/YN2ocRzwhY0/KJOegaDIep8J
25 | // NOTE: DO NOT upgrade to 1.2.2 since it is incompatiable and will cause tests fail
26 | "io.spray" % "spray-can" % "1.2.1",
27 | "io.spray" % "spray-routing" % "1.2.1",
28 | "io.spray" % "spray-client" % "1.2.1",
29 | yammerDeps
30 | ) ++ yodaDeps
31 |
32 | lazy val sparkDeps = Seq(
33 | "org.apache.spark" %% "spark-core" % "1.1.1" % "provided",
34 | // Force netty version. This avoids some Spark netty dependency problem.
35 | "io.netty" % "netty" % "3.6.6.Final"
36 | )
37 |
38 | lazy val slickDeps = Seq(
39 | "com.typesafe.slick" %% "slick" % "2.0.2-RC1",
40 | "com.h2database" % "h2" % "1.3.170",
41 | "mysql" % "mysql-connector-java" % "5.1.31",
42 | "org.mariadb.jdbc" % "mariadb-java-client" % "1.1.7"
43 | )
44 |
45 | lazy val logbackDeps = Seq(
46 | "ch.qos.logback" % "logback-classic" % "1.0.7"
47 | )
48 |
49 | lazy val coreTestDeps = Seq(
50 | "org.scalatest" %% "scalatest" % "1.9.1" % "test",
51 | "com.typesafe.akka" %% "akka-testkit" % "2.2.4" % "test",
52 | "io.spray" % "spray-testkit" % "1.2.0" % "test"
53 | )
54 |
55 |
56 | lazy val serverDeps = apiDeps ++ yodaDeps
57 | lazy val apiDeps = sparkDeps :+ typeSafeConfigDeps
58 | lazy val monitoringDeps = Seq(
59 | "com.codahale.metrics" % "metrics-core" % "3.0.1",
60 | // Upgrades version from 1.0.1 to 1.0.2 to support both HTTP and UDP.
61 | "org.coursera" % "metrics-datadog" % "1.0.2",
62 | "org.apache.httpcomponents" % "fluent-hc" % "4.3.2",
63 | "org.apache.httpcomponents" % "httpcore" % "4.3.2"
64 | )
65 |
66 | val repos = Seq(
67 | "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/",
68 | "sonatype snapshots" at "https://oss.sonatype.org/content/repositories/snapshots/",
69 | "spray repo" at "http://repo.spray.io"
70 | )
71 | }
72 |
--------------------------------------------------------------------------------
/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/test/spark.jobserver/stress/SingleContextJobStress.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.stress
2 |
3 | import akka.actor.{ActorSystem, Props}
4 | import akka.pattern.ask
5 | import akka.util.Timeout
6 | import com.typesafe.config.ConfigFactory
7 | import org.joda.time.DateTime
8 | import scala.concurrent.Await
9 | import spark.jobserver._
10 | import spark.jobserver.io.JobFileDAO
11 |
12 | /**
13 | * A stress test for launching many jobs within a job context
14 | * Launch using sbt> test:run
15 | * Watch with visualvm to see memory usage
16 | *
17 | * TODO(velvia): Turn this into an actual test. For now it's an app, requires manual testing.
18 | */
19 | object SingleContextJobStress extends App with TestJarFinder {
20 |
21 | import CommonMessages.JobResult
22 | import JobManagerActor._
23 | import scala.collection.JavaConverters._
24 | import scala.concurrent.duration._
25 |
26 | val jobDaoPrefix = "target/jobserver/"
27 | val config = ConfigFactory.parseString("""
28 | num-cpu-cores = 4 # Number of cores to allocate. Required.
29 | memory-per-node = 512m # Executor memory per node, -Xmx style eg 512m, 1G, etc.
30 | """)
31 |
32 | val system = ActorSystem("test", config)
33 | // Stuff needed for futures and Await
34 | implicit val ec = system
35 | implicit val ShortTimeout = Timeout(3 seconds)
36 |
37 | val jobDaoDir = jobDaoPrefix + DateTime.now.toString()
38 | val jobDaoConfig = ConfigFactory.parseMap(Map("spark.jobserver.filedao.rootdir" -> jobDaoDir).asJava)
39 | val dao = new JobFileDAO(jobDaoConfig)
40 |
41 | val jobManager = system.actorOf(Props(classOf[JobManagerActor], dao, "c1", "local[4]", config, false))
42 |
43 | private def uploadJar(jarFilePath: String, appName: String) {
44 | val bytes = scala.io.Source.fromFile(jarFilePath, "ISO-8859-1").map(_.toByte).toArray
45 | dao.saveJar(appName, DateTime.now, bytes)
46 | }
47 |
48 | private val demoJarPath = testJar.getAbsolutePath
49 | private val demoJarClass = "spark.jobserver.WordCountExample"
50 | private val emptyConfig = ConfigFactory.parseString("")
51 |
52 | // Create the context
53 | val res1 = Await.result(jobManager ? Initialize, 3 seconds)
54 | assert(res1.getClass == classOf[Initialized])
55 |
56 | uploadJar(demoJarPath, "demo1")
57 |
58 | // Now keep running this darn test ....
59 | var numJobs = 0
60 | val startTime = System.currentTimeMillis()
61 |
62 | while (true) {
63 | val f = jobManager ? StartJob("demo1", demoJarClass, emptyConfig, Set(classOf[JobResult]))
64 | Await.result(f, 3 seconds) match {
65 | case JobResult(info, Some(m)) =>
66 | numJobs += 1
67 | if (numJobs % 100 == 0) {
68 | val elapsed = System.currentTimeMillis() - startTime
69 | println("%d jobs finished in %f seconds".format(numJobs, elapsed / 1000.0))
70 | }
71 | case x =>
72 | println("Some error occurred: " + x)
73 | sys.exit(1)
74 | }
75 | // Thread sleep 1000
76 | }
77 | }
78 |
--------------------------------------------------------------------------------
/akka-app/src/ooyala.common.akka/web/JsonUtils.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka.web
2 |
3 | import spray.json._
4 | import spray.json.DefaultJsonProtocol._
5 |
6 |
7 | /**
8 | * JSON Serialization utilities for spray-json
9 | * Most of this is necessary because Spray-json doesn't deal well with Any types.
10 | */
11 | object JsonUtils {
12 | // Allows the conversion of flexible Maps that hold ints, strings, lists, maps
13 | // Note that this implicit conversion will only apply in this scope....
14 | // we have to be careful to make implicits that convert Any no wider in scope than needed
15 | implicit object AnyJsonFormat extends JsonFormat[Any] {
16 | def write(x: Any): JsValue = x match {
17 | case n: Int => JsNumber(n)
18 | case l: Long => JsNumber(l)
19 | case d: Double => JsNumber(d)
20 | case f: Float => JsNumber(f.toDouble)
21 | case s: String => JsString(s)
22 | case x: Seq[_] => seqFormat[Any].write(x)
23 | // Get the type of map keys from the first key, translate the rest the same way
24 | case m: Map[_, _] => if (m.isEmpty) {
25 | // Translates an emtpy map
26 | JsObject(Map[String, JsValue]())
27 | } else {
28 | m.keys.head match {
29 | case sym: Symbol =>
30 | val map = m.asInstanceOf[Map[Symbol, _]]
31 | val pairs = map.map { case (sym, v) => (sym.name -> write(v))}
32 | JsObject(pairs)
33 | case s: String => mapFormat[String, Any].write(m.asInstanceOf[Map[String, Any]])
34 | case a: Any =>
35 | val map = m.asInstanceOf[Map[Any, _]]
36 | val pairs = map.map { case (sym, v) => (sym.toString -> write(v))}
37 | JsObject(pairs)
38 | }
39 | }
40 | case a: Array[_] => seqFormat[Any].write(a.toSeq)
41 | case true => JsTrue
42 | case false => JsFalse
43 | case p: Product => seqFormat[Any].write(p.productIterator.toSeq)
44 | case null => JsNull
45 | case x => JsString(x.toString)
46 | }
47 | def read(value: JsValue): Any = value match {
48 | case JsNumber(n) => n.intValue()
49 | case JsString(s) => s
50 | case a: JsArray => listFormat[Any].read(value)
51 | case o: JsObject => mapFormat[String, Any].read(value)
52 | case JsTrue => true
53 | case JsFalse => false
54 | case x => deserializationError("Do not understand how to deserialize " + x)
55 | }
56 | }
57 |
58 | def mapToJson(map: Map[String, Any], compact: Boolean = true): String = {
59 | val jsonAst = map.toJson
60 | if (compact) jsonAst.compactPrint else jsonAst.prettyPrint
61 | }
62 |
63 | def listToJson(list: Seq[Any], compact: Boolean = true): String = {
64 | val jsonAst = list.toJson
65 | if (compact) jsonAst.compactPrint else jsonAst.prettyPrint
66 | }
67 |
68 | def mapFromJson(json: String): Map[String, Any] = json.asJson.convertTo[Map[String, Any]]
69 |
70 | def listFromJson(json: String): Seq[Any] = json.asJson.convertTo[Seq[Any]]
71 | }
72 |
--------------------------------------------------------------------------------
/akka-app/src/ooyala.common.akka/metrics/DatadogConfigParser.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka.metrics
2 |
3 | import collection.JavaConverters._
4 | import com.typesafe.config.Config
5 | import java.net.InetAddress
6 | import org.slf4j.LoggerFactory
7 | import scala.util.{Try, Success}
8 |
9 | /**
10 | * Datadog configuration
11 | *
12 | * @constructor create a new configuration for datadog reporting
13 | * @param hostName host name used for Datadog reporting
14 | * @param agentPort datadog agent UDP port
15 | * @param apiKey api key used for datadog reporting
16 | * @param tags the list of tags for datadog reporting
17 | * @param durationInSeconds durition in seconds between two Datadog reports
18 | */
19 | case class DatadogConfig(hostName: Option[String],
20 | agentPort: Option[Int],
21 | apiKey: Option[String],
22 | tags: Option[List[String]],
23 | durationInSeconds: Long = 30L)
24 |
25 | /**
26 | * Configuration parser for Datadog reporting
27 | */
28 | object DatadogConfigParser {
29 | private val logger = LoggerFactory.getLogger(getClass)
30 | // The Datadog configuraiton path inside a job server configuration file
31 | private val datadogConfigPath = "spark.jobserver.metrics.datadog"
32 |
33 | /**
34 | * Parses a configuration for Datadog reporting
35 | *
36 | * Parses the reporting host name, datadog agent port, api key, tags, and duration from the
37 | * Datadog configuration section. If the host name is not set, sets it to the local host name.
38 | *
39 | * Example config setting in spark.jobserver.metrics.datadog
40 | * spark.jobserver.metrics.datadog {
41 | * hostname = example
42 | * agentport = 8125
43 | * apikey = example
44 | * tags = ["tag1","tag2",...]
45 | * duration = 100
46 | * }
47 | *
48 | * @param config a configuraiton that contains a Datadog configuration section
49 | * @return a configuration for Datadog reporting
50 | */
51 | def parse(config: Config): DatadogConfig = {
52 | // Parses the host name, datadog agent port, api key, and tags.
53 | var hostName = Try(Option(config.getString(datadogConfigPath + ".hostname"))).getOrElse(None)
54 | val agentPort = Try(Option(config.getInt(datadogConfigPath + ".agentport"))).getOrElse(None)
55 | val apiKey = Try(Option(config.getString(datadogConfigPath + ".apikey"))).getOrElse(None)
56 | val tags = Try(Option(config.getStringList(datadogConfigPath + ".tags").asScala.toList))
57 | .getOrElse(None)
58 |
59 | if (hostName.isEmpty) {
60 | // Uses local host name if the host name is not set.
61 | hostName = Try(Option(InetAddress.getLocalHost.getCanonicalHostName)).getOrElse(None)
62 | }
63 |
64 | // Parses the Datadog reporting duration
65 | Try(config.getLong(datadogConfigPath + ".duration")) match {
66 | case Success(duration) =>
67 | DatadogConfig(hostName, agentPort, apiKey, tags, duration)
68 | case _ =>
69 | DatadogConfig(hostName, agentPort, apiKey, tags)
70 | }
71 | }
72 | }
--------------------------------------------------------------------------------
/akka-app/test/ooyala.common.akka/web/CommonRoutesSpec.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka.web
2 |
3 | import java.util.concurrent.TimeUnit
4 |
5 | import org.scalatest.FunSpec
6 | import org.scalatest.matchers.ShouldMatchers
7 | import spray.testkit.ScalatestRouteTest
8 |
9 | import spray.http.StatusCodes._
10 | import com.yammer.metrics.Metrics
11 | import com.yammer.metrics.core.Gauge
12 |
13 | class CommonRoutesSpec extends FunSpec with ShouldMatchers with ScalatestRouteTest with CommonRoutes {
14 | def actorRefFactory = system
15 |
16 |
17 | val metricCounter = Metrics.newCounter(getClass, "test-counter")
18 | val metricMeter = Metrics.newMeter(getClass, "test-meter", "requests", TimeUnit.SECONDS)
19 | val metricHistogram = Metrics.newHistogram(getClass, "test-hist")
20 | val metricTimer = Metrics.newTimer(getClass, "test-timer", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)
21 | val metricGauge = Metrics.newGauge(getClass, "test-gauge", new Gauge[Int] {
22 | def value() = 10
23 | })
24 |
25 | val counterMap = Map("type" -> "counter", "count" -> 0)
26 | val gaugeMap = Map("type" -> "gauge", "value" -> 10)
27 |
28 | val meterMap = Map("type" -> "meter", "units" -> "seconds", "count" -> 0, "mean" -> 0.0,
29 | "m1" -> 0.0, "m5" -> 0.0, "m15" -> 0.0)
30 | val histMap = Map("type" -> "histogram", "median" -> 0.0, "p75" -> 0.0, "p95" -> 0.0,
31 | "p98" -> 0.0, "p99" -> 0.0, "p999" -> 0.0)
32 | val timerMap = Map("type" -> "timer", "rate" -> (meterMap - "type"),
33 | "duration" -> (histMap ++ Map("units" -> "milliseconds") - "type"))
34 |
35 | describe("/metricz route") {
36 | it("should report all metrics") {
37 | Get("/metricz") ~> commonRoutes ~> check {
38 | status === OK
39 |
40 | val metricsMap = JsonUtils.mapFromJson(entityAs[String])
41 | val classMetrics = metricsMap(getClass.getName).asInstanceOf[Map[String, Any]]
42 |
43 | classMetrics.keys.toSet should equal (Set("test-counter", "test-meter", "test-hist", "test-timer", "test-gauge"))
44 | classMetrics("test-counter") should equal (counterMap)
45 | classMetrics("test-meter") should equal (meterMap)
46 | classMetrics("test-hist") should equal (histMap)
47 | classMetrics("test-timer") should equal (timerMap)
48 | }
49 | }
50 | }
51 |
52 | describe("metrics serializer") {
53 | it("should serialize all metrics") {
54 | val flattenedMap = MetricsSerializer.asFlatMap()
55 |
56 | List("test-meter", "test-counter", "test-timer", "test-gauge", "test-hist") foreach { metricName =>
57 | flattenedMap.keys should contain("ooyala.common.akka.web.CommonRoutesSpec." + metricName)
58 | }
59 |
60 | flattenedMap("ooyala.common.akka.web.CommonRoutesSpec.test-meter") should equal(meterMap)
61 | flattenedMap("ooyala.common.akka.web.CommonRoutesSpec.test-counter") should equal(counterMap)
62 | flattenedMap("ooyala.common.akka.web.CommonRoutesSpec.test-hist") should equal(histMap)
63 | flattenedMap("ooyala.common.akka.web.CommonRoutesSpec.test-timer") should equal(timerMap)
64 | flattenedMap("ooyala.common.akka.web.CommonRoutesSpec.test-gauge") should equal(gaugeMap)
65 | }
66 | }
67 | }
68 |
--------------------------------------------------------------------------------
/akka-app/test/ooyala.common.akka/web/JsonUtilsSpec.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka.web
2 |
3 | import org.joda.time.DateTime
4 | import org.parboiled.errors.ParsingException
5 | import org.scalatest.FunSpec
6 | import org.scalatest.matchers.ShouldMatchers
7 |
8 | class JsonUtilsSpec extends FunSpec with ShouldMatchers {
9 | import spray.json._
10 | import spray.json.DefaultJsonProtocol._
11 |
12 | val Dt1 = "2012-09-09T04:18:13.001Z"
13 | val Dt2 = "2012-09-09T04:18:13.002Z"
14 |
15 | describe("JSON conversion") {
16 | it("should generate proper JSON for list of maps") {
17 | val batch = Seq(Map(Dt1 -> Map("ipaddr" -> "1.2.3.4")),
18 | Map(Dt2 -> Map("guid" -> "xyz")))
19 | val expected = """[{"2012-09-09T04:18:13.001Z":{"ipaddr":"1.2.3.4"}},""" +
20 | """{"2012-09-09T04:18:13.002Z":{"guid":"xyz"}}]"""
21 | JsonUtils.listToJson(batch) should equal (expected)
22 | }
23 |
24 | it("should generate map from JSON") {
25 | val json = """[{"2012-09-09T04:18:13.002Z":{"ipaddr":"1.2.3.5"}},
26 | {"2012-09-09T04:18:13.001Z":{"guid":"abc"}}]"""
27 | val batch = Seq(Map(Dt2 -> Map("ipaddr" -> "1.2.3.5")),
28 | Map(Dt1 -> Map("guid" -> "abc")))
29 | JsonUtils.listFromJson(json) should equal (batch)
30 | }
31 |
32 | it("should serialize empty maps") {
33 | val expected1 = """{"a":1,"b":{}}"""
34 | import JsonUtils._
35 | // Serializes a simple map that contains an emtpy map.
36 | Map("a" -> 1, "b" -> Map()).toJson.compactPrint should equal (expected1)
37 |
38 | val expected2 = """{"a":1,"b":{"a1":1,"b1":{}}}"""
39 | // Serializes a map that embeds an empty map in a deeper level.
40 | Map("a" -> 1, "b" -> Map("a1" -> 1, "b1" -> Map())).toJson
41 | .compactPrint should equal (expected2)
42 | }
43 |
44 | it("should serialize some other types") {
45 | val expected1 = """{"1":[1,2,3]}"""
46 | import JsonUtils._
47 | Map("1" -> Array(1, 2, 3): (String, Any)).toJson.compactPrint should equal (expected1)
48 |
49 | val expected2 = """{"1":[1,2,"b"]}"""
50 | Map("1" -> (1, 2, "b")).toJson.compactPrint should equal (expected2)
51 | }
52 |
53 | it("should serialize unknown types to their string representations") {
54 | val expected = "[1,2,\"" + Dt1 + "\"]"
55 | import JsonUtils._
56 | Seq(1, 2, DateTime.parse(Dt1)).toJson.compactPrint should equal (expected)
57 | }
58 |
59 | it("should throw exception for invalid JSON") {
60 | val badJson1 = """{123: 456}""" // objects must have string keys
61 | val badJson2 = """["abc]""" // unbalanced quotes
62 | intercept[ParsingException](JsonUtils.listFromJson(badJson1))
63 | intercept[ParsingException](JsonUtils.listFromJson(badJson2))
64 | intercept[ParsingException](JsonUtils.mapFromJson(badJson1))
65 | intercept[ParsingException](JsonUtils.mapFromJson(badJson2))
66 | }
67 |
68 | it("should throw exception for valid JSON that doesn't conform to expected type") {
69 | intercept[DeserializationException](JsonUtils.listFromJson("""{"1": 2}"""))
70 | intercept[DeserializationException](JsonUtils.mapFromJson("""["123"]"""))
71 | }
72 | }
73 | }
--------------------------------------------------------------------------------
/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/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/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 | }
--------------------------------------------------------------------------------
/akka-app/src/ooyala.common.akka/metrics/CompactMetricsWriter.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka.metrics
2 |
3 | import com.yammer.metrics.core._
4 | import com.yammer.metrics.Metrics
5 | import org.slf4j.Logger
6 | import java.util.concurrent.TimeUnit
7 | import com.yammer.metrics.stats.Snapshot
8 |
9 |
10 | /**
11 | * Writes out metrics in a form suitable for writing to log files. For example a counter will be written
12 | * out as
13 | *
14 | * ooyala.common.akka.example.SomeActor.pending-futures(count = 20)
15 | *
16 | */
17 | class CompactMetricsWriter(private val log: Logger) extends MetricProcessor[Logger] {
18 |
19 | import collection.JavaConverters._
20 |
21 | def process(registry: MetricsRegistry = Metrics.defaultRegistry()) {
22 | registry.allMetrics().asScala.foreach {
23 | case (metricName, metricsObject) => metricsObject.processWith(this, metricName, log)
24 | }
25 | }
26 |
27 | def processMeter(name: MetricName, meter: Metered, logger: Logger) {
28 | logger.info(processMetric(name) { sb => renderMeter(meter, sb) })
29 | }
30 |
31 | def processCounter(name: MetricName, counter: Counter, context: Logger) {
32 | context.info(processMetric(name) { sb =>
33 | sb.append("count = " + counter.count())
34 | })
35 | }
36 |
37 | def processHistogram(name: MetricName, histogram: Histogram, logger: Logger) {
38 | logger.info(processMetric(name) { sb => renderHistogram(histogram.getSnapshot, sb)})
39 | }
40 |
41 | def processTimer(name: MetricName, timer: Timer, logger: Logger) {
42 | logger.info(processMetric(name) { sb =>
43 | renderHistogram(timer.getSnapshot, sb, abbrev(timer.durationUnit()))
44 | sb.append(", ")
45 | renderMeter(timer, sb)
46 | })
47 | }
48 |
49 | def processGauge(name: MetricName, gauge: Gauge[_], context: Logger) {
50 | context.info(processMetric(name) { sb =>
51 | sb.append("gauge = " + gauge.value())
52 | })
53 | }
54 |
55 | private def processMetric(metricName: MetricName)(func: (StringBuilder) => Unit): String = {
56 | val sb = new StringBuilder()
57 | sb.append(metricName.getGroup + "." + metricName.getType + "." + metricName.getName + "(")
58 | func(sb)
59 | sb.append(")")
60 | sb.toString()
61 | }
62 |
63 | private def renderHistogram(snapshot: Snapshot, sb: StringBuilder, unit: String = "") {
64 | sb.append("median = " + "%2.2f%s".format(snapshot.getMedian, unit))
65 | sb.append(", p75 = " + "%2.2f%s".format(snapshot.get75thPercentile(), unit))
66 | sb.append(", p95 = " + "%2.2f%s".format(snapshot.get95thPercentile(), unit))
67 | sb.append(", p98 = " + "%2.2f%s".format(snapshot.get98thPercentile(), unit))
68 | sb.append(", p99 = " + "%2.2f%s".format(snapshot.get99thPercentile(), unit))
69 | sb.append(", p999 = " + "%2.2f%s".format(snapshot.get99thPercentile(), unit))
70 | }
71 |
72 | private def renderMeter(meter: Metered, sb: StringBuilder) {
73 | val unit = abbrev(meter.rateUnit())
74 |
75 | sb.append("count = " + meter.count)
76 | sb.append(", mean = " + "%2.2f/%s".format(meter.meanRate(), unit))
77 | sb.append(", m1 = " + "%2.2f/%s".format(meter.oneMinuteRate(), unit))
78 | sb.append(", m5 = " + "%2.2f/%s".format(meter.fiveMinuteRate(), unit))
79 | sb.append(", m15 = " + "%2.2f/%s".format(meter.fifteenMinuteRate(), unit))
80 | }
81 |
82 | private def abbrev(timeUnit: TimeUnit) = {
83 | timeUnit match {
84 | case TimeUnit.NANOSECONDS => "ns"
85 | case TimeUnit.MICROSECONDS => "us"
86 | case TimeUnit.MILLISECONDS => "ms"
87 | case TimeUnit.SECONDS => "s"
88 | case TimeUnit.MINUTES => "m"
89 | case TimeUnit.HOURS => "h"
90 | case TimeUnit.DAYS => "d"
91 | case _ =>
92 | throw new IllegalArgumentException("Unrecognized TimeUnit: " + timeUnit)
93 | }
94 |
95 | }
96 | }
97 |
--------------------------------------------------------------------------------
/akka-app/src/ooyala.common.akka/metrics/MetricsWrapper.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka.metrics
2 |
3 | import collection.JavaConverters._
4 | import com.codahale.metrics._
5 | import java.util.concurrent.TimeUnit
6 | import org.coursera.metrics.datadog.DatadogReporter
7 | import org.coursera.metrics.datadog.DatadogReporter.Expansion
8 | import org.coursera.metrics.datadog.transport.{HttpTransport, Transport, UdpTransport}
9 | import org.slf4j.LoggerFactory
10 | import scala.util.Try
11 |
12 | object MetricsWrapper {
13 | private val logger = LoggerFactory.getLogger(getClass)
14 | val registry: MetricRegistry = new MetricRegistry
15 | private var shutdownHook: Thread = null
16 |
17 | // Registers JVM metrics for monitoring
18 | JvmMetricsWrapper.registerJvmMetrics(registry)
19 |
20 | def startDatadogReporter(config: DatadogConfig): Unit = {
21 | val transportOpt: Option[Transport] = config.agentPort.map {
22 | port =>
23 | logger.debug("Datadog reporter: datadog agent port - " + port)
24 | new UdpTransport.Builder().withPort(port).build
25 | } orElse config.apiKey.map {
26 | apiKey => new HttpTransport.Builder().withApiKey(apiKey).build
27 | }
28 |
29 | transportOpt match {
30 | case Some(transport) =>
31 | val datadogReporterBuilder = DatadogReporter.forRegistry(registry)
32 |
33 | // Adds host name
34 | config.hostName match {
35 | case Some(hostName) =>
36 | logger.debug("Datadog reporter: hostname - " + hostName)
37 | datadogReporterBuilder.withHost(hostName)
38 | case _ =>
39 | logger.info("No host name provided, won't report host name to datadog.")
40 | }
41 |
42 | // Adds tags if provided
43 | config.tags.foreach {
44 | tags =>
45 | logger.debug("Datadog reporter: tags - " + tags)
46 | datadogReporterBuilder.withTags(tags.asJava)
47 | }
48 |
49 | val datadogReporter = datadogReporterBuilder
50 | .withTransport(transport)
51 | .withExpansions(Expansion.ALL)
52 | .build
53 |
54 | shutdownHook = new Thread {
55 | override def run {
56 | datadogReporter.stop
57 | }
58 | }
59 |
60 | // Start the reporter and set up shutdown hooks
61 | datadogReporter.start(config.durationInSeconds, TimeUnit.SECONDS)
62 | Runtime.getRuntime.addShutdownHook(shutdownHook)
63 |
64 | logger.info("Datadog reporter started.")
65 | case _ =>
66 | logger.info("No transport available, Datadog reporting not started.")
67 | }
68 | }
69 |
70 | def newGauge[T](klass: Class[_], name: String, metric: => T): Gauge[T] = {
71 | val metricName = MetricRegistry.name(klass, name)
72 | val gauge = Try(registry.register(metricName, new Gauge[T] {
73 | override def getValue(): T = metric
74 | })).map { g => g } recover { case _ =>
75 | registry.getGauges.get(metricName)
76 | }
77 | gauge.get.asInstanceOf[Gauge[T]]
78 | }
79 |
80 | def newCounter(klass: Class[_], name: String): Counter =
81 | registry.counter(MetricRegistry.name(klass, name))
82 |
83 | def newTimer(klass: Class[_], name: String): Timer =
84 | registry.timer(MetricRegistry.name(klass, name))
85 |
86 | def newHistogram(klass: Class[_], name: String): Histogram =
87 | registry.histogram(MetricRegistry.name(klass, name))
88 |
89 | def newMeter(klass: Class[_], name: String): Meter =
90 | registry.meter(MetricRegistry.name(klass, name))
91 |
92 | def getRegistry: MetricRegistry = {
93 | return registry
94 | }
95 |
96 | def shutdown = {
97 | if (shutdownHook != null) {
98 | Runtime.getRuntime.removeShutdownHook(shutdownHook)
99 | shutdownHook.run
100 | }
101 | }
102 | }
--------------------------------------------------------------------------------
/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/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/test/spark.jobserver/io/JobSqlDAOJdbcConfigSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.io
2 |
3 | import com.typesafe.config.ConfigFactory
4 | import org.scalatest.FunSpec
5 | import org.scalatest.matchers.ShouldMatchers
6 | import spark.jobserver.TestJarFinder
7 |
8 |
9 | class JobSqlDAOJdbcConfigSpec extends TestJarFinder with FunSpec with ShouldMatchers {
10 |
11 | describe("parse MariaDB config") {
12 | val configStr =
13 | """
14 | spark.jobserver.sqldao {
15 | rootdir = /tmp/spark-job-server-test/sqldao/data
16 |
17 | mariadb {
18 | url = "jdbc:mariadb://localhost:3306/test"
19 | user = test
20 | password = test
21 | }
22 | }
23 | """
24 |
25 | it("should parse a valid MariaDB config") {
26 | val config = ConfigFactory.parseString(configStr)
27 | val mariadbConfig = MariaDbConfigParser.parse(config)
28 |
29 | mariadbConfig.isDefined should equal(true)
30 | mariadbConfig.get.user should equal("test")
31 | mariadbConfig.get.password should equal("test")
32 | }
33 |
34 | it("should fail to parse a MariaDB config") {
35 | // An invalid MariaDB config that has an invalid JDBC connection url prefix
36 | val invalidConfigStr = configStr.replace("jdbc:mariadb:", "jdbc:tiffanydb:")
37 | val config = ConfigFactory.parseString(invalidConfigStr)
38 | val mariadbConfig = MariaDbConfigParser.parse(config)
39 |
40 | mariadbConfig.isDefined should equal(false)
41 | }
42 | }
43 |
44 | describe("parse MySQL config") {
45 | val configStr =
46 | """
47 | spark.jobserver.sqldao {
48 | rootdir = /tmp/spark-job-server-test/sqldao/data
49 |
50 | mysql {
51 | url = "jdbc:mysql://localhost:3306/test"
52 | user = test
53 | password = test
54 | }
55 | }
56 | """
57 |
58 | it("should parse a valid MySQL config") {
59 | val config = ConfigFactory.parseString(configStr)
60 | val mysqlConfig = MySqlConfigParser.parse(config)
61 |
62 | mysqlConfig.isDefined should equal(true)
63 | mysqlConfig.get.user should equal("test")
64 | mysqlConfig.get.password should equal("test")
65 | }
66 |
67 | it("should fail to parse a MySQL config") {
68 | // An invalid MySQL config that has an invalid JDBC connection url prefix
69 | val invalidConfigStr = configStr.replace("jdbc:mysql:", "jdbc:sql:")
70 | val config = ConfigFactory.parseString(invalidConfigStr)
71 | val mysqlConfig = MySqlConfigParser.parse(config)
72 |
73 | mysqlConfig.isDefined should equal(false)
74 | }
75 | }
76 |
77 | describe("parse H2 config") {
78 | val configStr =
79 | """
80 | spark.jobserver.sqldao {
81 | rootdir = /tmp/spark-job-server-test/sqldao/data
82 |
83 | h2 {
84 | url = "jdbc:h2:file:/tmp/test"
85 | user = test
86 | password = test
87 | }
88 | }
89 | """
90 |
91 | it("should parse a valid H2 config") {
92 | val config = ConfigFactory.parseString(configStr)
93 | val h2Config = H2ConfigParser.parse(config)
94 |
95 | h2Config.isDefined should equal(true)
96 | h2Config.get.user should equal("test")
97 | h2Config.get.password should equal("test")
98 | }
99 |
100 | it("should fail to parse H2 config") {
101 | // An invalid H2 config that has an invalid JDBC connection url prefix
102 | val invalidConfigStr = configStr.replace("jdbc:h2:", "jdbc:hh2:")
103 | val config = ConfigFactory.parseString(invalidConfigStr)
104 | val h2Config = H2ConfigParser.parse(config)
105 |
106 | h2Config.isDefined should equal(false)
107 | }
108 | }
109 |
110 | describe("parse default config") {
111 | it("should return a default H2 config") {
112 | val config = ConfigFactory.parseString("")
113 | val jdbcConfigOpt = JdbcConfigParserFactory.parse(config)
114 | val jdbcConfig = jdbcConfigOpt.getOrElse(H2ConfigParser.defaultConfig)
115 |
116 | jdbcConfigOpt should equal(None)
117 | jdbcConfig.url.startsWith("jdbc:h2:") should equal(true)
118 | }
119 | }
120 | }
--------------------------------------------------------------------------------
/job-server/src/main/resources/html/index.html:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 | Spark Jobserver UI
5 |
6 |
7 |
8 |
9 |
10 |
11 |
12 |
13 |
14 |
15 |
16 |
19 |
24 |
25 |
26 |
27 |
Running Jobs
28 |
29 |
30 |
31 |
32 | Id
33 | Classpath
34 | Context
35 | Start Time
36 | Duration
37 |
38 |
39 |
40 |
41 |
42 |
43 |
44 |
45 |
Completed Jobs
46 |
47 |
48 |
49 |
50 | Id
51 | Classpath
52 | Context
53 | Start Time
54 | Duration
55 |
56 |
57 |
58 |
59 |
60 |
61 |
62 |
63 |
Failed Jobs
64 |
65 |
66 |
67 |
68 | Id
69 | Classpath
70 | Context
71 | Start Time
72 | Duration
73 |
74 |
75 |
76 |
77 |
78 |
79 |
80 |
81 |
82 |
83 |
84 |
85 |
86 |
87 |
88 | Name
89 |
90 |
91 |
92 |
93 |
94 |
95 |
96 |
97 |
98 |
99 |
100 |
101 |
102 |
103 |
104 | Name
105 | Deployment Time
106 |
107 |
108 |
109 |
110 |
111 |
112 |
113 |
114 |
115 |
116 |
117 |
--------------------------------------------------------------------------------
/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/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 |
--------------------------------------------------------------------------------
/akka-app/test/ooyala.common.akka/metrics/DatadogConfigParserSpec.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka.metrics
2 |
3 | import com.typesafe.config.ConfigFactory
4 | import java.net.InetAddress
5 | import org.scalatest.FunSpec
6 | import org.scalatest.matchers.ShouldMatchers
7 |
8 | class DatadogConfigParserSpec extends FunSpec with ShouldMatchers {
9 |
10 | describe("parses Datadog config") {
11 | it("should parse a valid config") {
12 | val configStr =
13 | """
14 | spark.jobserver.metrics.datadog {
15 | hostname = "test"
16 | apikey = "12345"
17 | duration = 10
18 | }
19 | """
20 | val config = ConfigFactory.parseString(configStr)
21 | val datadogConfig = DatadogConfigParser.parse(config)
22 |
23 | val hostName = datadogConfig.hostName
24 | hostName.isDefined should equal(true)
25 | hostName.get should equal("test")
26 |
27 | val apiKey = datadogConfig.apiKey
28 | apiKey.isDefined should equal(true)
29 | apiKey.get should equal("12345")
30 |
31 | datadogConfig.durationInSeconds should equal(10L)
32 | }
33 |
34 | it("should return local host name and an api key") {
35 | // Omits host name
36 | val configStr =
37 | """
38 | spark.jobserver.metrics.datadog {
39 | apikey = "12345"
40 | duration = 10
41 | }
42 | """
43 |
44 | val config = ConfigFactory.parseString(configStr)
45 | val datadogConfig = DatadogConfigParser.parse(config)
46 |
47 | val hostName = datadogConfig.hostName
48 | hostName.isDefined should equal(true)
49 | // When host name isn't defined in config file, local host name should be returned
50 | hostName.get should equal(InetAddress.getLocalHost.getCanonicalHostName)
51 |
52 | val apiKey = datadogConfig.apiKey
53 | apiKey.isDefined should equal(true)
54 | apiKey.get should equal("12345")
55 |
56 | datadogConfig.durationInSeconds should equal(10L)
57 | }
58 |
59 | it ("should return only local host name") {
60 | // Omits host name and api key
61 | val configStr =
62 | """
63 | spark.jobserver.metrics.datadog {
64 | }
65 | """
66 |
67 | val config = ConfigFactory.parseString(configStr)
68 | val datadogConfig = DatadogConfigParser.parse(config)
69 |
70 | val hostName = datadogConfig.hostName
71 | hostName.isDefined should equal(true)
72 | hostName.get should equal(InetAddress.getLocalHost.getCanonicalHostName)
73 |
74 | datadogConfig.apiKey.isDefined should equal(false)
75 | }
76 |
77 | it ("should parse a valid list of tags") {
78 | val configStr =
79 | """
80 | spark.jobserver.metrics.datadog {
81 | tags = ["test version", "has tags"],
82 | }
83 | """
84 | val config = ConfigFactory.parseString(configStr)
85 | val datadogConfig = DatadogConfigParser.parse(config)
86 |
87 | val tags = datadogConfig.tags
88 | tags.isDefined should equal(true)
89 | tags.get should equal(List("test version", "has tags"))
90 | }
91 |
92 | it ("should parse no tags") {
93 | val configStr =
94 | """
95 | spark.jobserver.metrics.datadog {
96 | }
97 | """
98 | val config = ConfigFactory.parseString(configStr)
99 | val datadogConfig = DatadogConfigParser.parse(config)
100 |
101 | val tags = datadogConfig.tags
102 | tags.isDefined should equal(false)
103 | }
104 |
105 | it ("should parse a valid datadog agent port") {
106 | val configStr =
107 | """
108 | spark.jobserver.metrics.datadog {
109 | agentport = 9999
110 | }
111 | """
112 | val config = ConfigFactory.parseString(configStr)
113 | val datadogConfig = DatadogConfigParser.parse(config)
114 |
115 | val agentPort = datadogConfig.agentPort
116 | agentPort.isDefined should equal(true)
117 | agentPort.get should equal(9999)
118 | }
119 |
120 | it ("should parse no datadog agent port") {
121 | val configStr =
122 | """
123 | spark.jobserver.metrics.datadog {
124 | }
125 | """
126 | val config = ConfigFactory.parseString(configStr)
127 | val datadogConfig = DatadogConfigParser.parse(config)
128 |
129 | val agentPort = datadogConfig.agentPort
130 | agentPort.isDefined should equal(false)
131 | }
132 | }
133 | }
--------------------------------------------------------------------------------
/akka-app/src/ooyala.common.akka/web/CommonRoutes.scala:
--------------------------------------------------------------------------------
1 | package ooyala.common.akka.web
2 |
3 | import spray.routing.HttpService
4 | import com.yammer.metrics.core._
5 | import com.yammer.metrics.Metrics
6 |
7 | /**
8 | * Defines a couple common Spray routes for metrics, status, debugging
9 | * * /metricz - dumps out all application metrics
10 | * * /statusz - dumps out GIT status of the running code
11 | */
12 | trait CommonRoutes extends HttpService {
13 | val commonRoutes = {
14 | get {
15 | path("metricz") {
16 | // TODO: Support option to return only certain metrics classes, or turn off pretty printing
17 | complete { MetricsSerializer.serialize() }
18 | } ~
19 | path("statusz") {
20 | getFromFile("statusz.html")
21 | }
22 | }
23 | }
24 | }
25 |
26 |
27 | /**
28 | * Serializes all the Metrics objects into JSON string
29 | */
30 | object MetricsSerializer {
31 | import collection.JavaConverters._
32 |
33 | def serialize(registry: MetricsRegistry = Metrics.defaultRegistry(),
34 | classPrefix: String = null): String = {
35 | val map = asGroupedMap(registry, classPrefix)
36 | JsonUtils.mapToJson(map, compact = false)
37 | }
38 |
39 | /** Returns all the metrics, grouped by the class name
40 | *
41 | * @param registry default registry if not specified
42 | * @param classPrefix only return metrics of this type
43 | * @return Map(className -> (metricName -> MetricMap))
44 | */
45 | def asGroupedMap(registry: MetricsRegistry = Metrics.defaultRegistry(),
46 | classPrefix: String = null): Map[String, Map[String, Map[String, Any]]] = {
47 | registry.groupedMetrics().asScala.flatMap {
48 | case (metricsClass, metricsBlob) =>
49 | if (classPrefix == null || metricsClass.startsWith(classPrefix)) {
50 | val innerMap = metricsBlob.asScala.flatMap {
51 | case (metricName, metric) =>
52 | try {
53 | Some(metricName.getName -> process(metric))
54 | } catch {
55 | case e: Exception =>
56 | None
57 | }
58 | }.toMap
59 | Some(metricsClass -> innerMap)
60 | } else { None }
61 | }.toMap
62 | }
63 |
64 |
65 | /** Returns all the metrics keyed by the full metric name */
66 | def asFlatMap(registry: MetricsRegistry = Metrics.defaultRegistry(),
67 | classPrefix: String = null): Map[String, Map[String, Any]] = {
68 |
69 | // TODO: There is a fair amount of code duplication here
70 | val metrics = registry.allMetrics().asScala
71 | metrics.flatMap {
72 | case (metricName, metricsBlob) =>
73 | try {
74 | Some(metricName.getGroup + "." + metricName.getType + "." + metricName.getName()
75 | -> process(metricsBlob))
76 | } catch {
77 | case e: Exception => None
78 | }
79 | }.toMap
80 | }
81 |
82 | private def process(metric: Metric): Map[String, Any] = {
83 | metric match {
84 | case c: Counter => Map("type" -> "counter", "count" -> c.count())
85 | case m: Meter => Map("type" -> "meter") ++ meterToMap(m)
86 | case g: Gauge[_] => Map("type" -> "gauge", "value" -> g.value())
87 | // For Timers, ignore the min/max/mean values, as they are for all time. We're just interested
88 | // in the recent (biased) histogram values.
89 | case h: Histogram => Map("type" -> "histogram") ++ histogramToMap(h)
90 | case t: Timer =>
91 | Map("type" -> "timer", "rate" -> meterToMap(t),
92 | "duration" -> (histogramToMap(t) ++ Map("units" -> t.durationUnit.toString.toLowerCase)))
93 |
94 | }
95 | }
96 |
97 | private def meterToMap(m: Metered) =
98 | Map("units" -> m.rateUnit.toString.toLowerCase,
99 | "count" -> m.count,
100 | "mean" -> m.meanRate,
101 | "m1" -> m.oneMinuteRate,
102 | "m5" -> m.fiveMinuteRate,
103 | "m15" -> m.fifteenMinuteRate)
104 |
105 | /** Extracts the histogram (Median, 75%, 95%, 98%, 99% 99.9%) values to a map */
106 | private def histogramToMap(h: Sampling) =
107 | Map("median" -> h.getSnapshot().getMedian(),
108 | "p75" -> h.getSnapshot().get75thPercentile(),
109 | "p95" -> h.getSnapshot().get95thPercentile(),
110 | "p98" -> h.getSnapshot().get98thPercentile(),
111 | "p99" -> h.getSnapshot().get99thPercentile(),
112 | "p999" -> h.getSnapshot().get999thPercentile())
113 | }
114 |
--------------------------------------------------------------------------------
/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/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/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/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 | |Applications:
67 | | 3 Running,
68 | | 67 Completed
69 | | ALIVE
70 | | ALIVE
71 | | DEAD
72 | | blah blah
73 | """.stripMargin)
74 | }
75 | case HttpRequest(GET, Uri.Path("/stop"), _, _, _) =>
76 | sender ! HttpResponse(entity = "Shutting down in 1 second ...")
77 | sender ! Http.Close
78 | context.system.scheduler.scheduleOnce(1.second) { context.system.shutdown() }
79 | }
80 | }
81 |
82 | class SparkWebUiActorSpec extends TestKit(SparkWebUiActorSpec.system) with ImplicitSender
83 | with FunSpec with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll {
84 |
85 | // Used in the asks (?) below to request info from contextSupervisor and resultActor
86 | implicit val ShortTimeout = Timeout(3 seconds)
87 |
88 | override def beforeAll(): Unit = {
89 | import akka.actor.{ActorSystem, Props}
90 | import akka.io.IO
91 | import spray.can.Http
92 |
93 | implicit val system = ActorSystem()
94 | // the handler actor replies to incoming HttpRequests
95 | val handler = system.actorOf(Props[SimpleHttpServer], name = "simpleHttpServer")
96 | IO(Http) ! Http.Bind(handler, interface = SparkWebUiActorSpec.sparkWebUrl, port = SparkWebUiActorSpec.sparkWebPort)
97 | }
98 |
99 | override def afterAll() {
100 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(SparkWebUiActorSpec.system)
101 |
102 | // close the web service
103 | implicit val system = ActorSystem("test")
104 | import system.dispatcher // execution context for futures below
105 |
106 | val pipeline: Future[SendReceive] =
107 | for (
108 | Http.HostConnectorInfo(connector, _) <-
109 | IO(Http) ? Http.HostConnectorSetup(SparkWebUiActorSpec.sparkWebUrl, port = SparkWebUiActorSpec.sparkWebPort)
110 | ) yield sendReceive(connector)
111 |
112 | val request = Get("/stop")
113 | pipeline.flatMap(_(request)) // async call
114 |
115 | }
116 |
117 | var actor: ActorRef = _
118 |
119 | before {
120 | actor = SparkWebUiActorSpec.system.actorOf(Props(classOf[SparkWebUiActor]), "spark-web-ui")
121 | }
122 |
123 | after {
124 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(actor)
125 | }
126 |
127 | describe("SparkWebUiActor") {
128 | it("should get worker info") {
129 | val future = actor ? GetWorkerStatus()
130 | val result = Await.result(future, ShortTimeout.duration).asInstanceOf[SparkWorkersInfo]
131 | result.alive should equal (2)
132 | result.dead should equal (1)
133 | }
134 | }
135 | }
--------------------------------------------------------------------------------
/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/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/test/spark.jobserver/io/JobDAOMetricsMonitorSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.io
2 |
3 | import com.codahale.metrics.MetricRegistry
4 | import com.typesafe.config.{ConfigFactory, ConfigRenderOptions}
5 | import ooyala.common.akka.metrics.{MetricsLevel, MetricsWrapper}
6 | import org.joda.time.DateTime
7 | import org.scalatest.{BeforeAndAfter, FunSpec}
8 | import org.scalatest.matchers.ShouldMatchers
9 | import spark.jobserver.InMemoryDAO
10 |
11 | class JobDAOMetricsMonitorSpec extends FunSpec with ShouldMatchers with BeforeAndAfter {
12 | private val jobId = "jobId"
13 | private val contextName = "contextName"
14 | private val appName = "appName"
15 | private val jarBytes = Array.fill[Byte](10)(1)
16 | private val jarInfo = JarInfo(appName, DateTime.now)
17 | private val classPath = "classPath"
18 | private val jobInfo = JobInfo(jobId, contextName, jarInfo, classPath, DateTime.now, None, None)
19 | private val jobConfig = ConfigFactory.parseString("{job-name=test}")
20 | private val contextConfig = ConfigFactory.parseString({"num-cpu=2"})
21 | private val dao = new InMemoryDAO
22 | private val basicMetrics = Array("saveJar", "saveJobInfo", "saveJobConfig", "saveContextConfig").map {
23 | name => MetricRegistry.name(classOf[JobDAOMetricsMonitor], name)
24 | }
25 | private val fineMetrics = Array("JarSize", "JobInfoSize", "JobConfigSize", "ContextConfigSize").map {
26 | name => MetricRegistry.name(classOf[JobDAOMetricsMonitor], name)
27 | }
28 |
29 | before {
30 | // Resets metrics
31 | basicMetrics.foreach {
32 | metricName =>
33 | MetricsWrapper.getRegistry.remove(metricName)
34 | MetricsWrapper.getRegistry.meter(metricName)
35 | }
36 | fineMetrics.foreach {
37 | metricName =>
38 | MetricsWrapper.getRegistry.remove(metricName)
39 | MetricsWrapper.getRegistry.histogram(metricName)
40 | }
41 | }
42 |
43 | describe("JobDAO metrics monitoring") {
44 | it("should increment basic metric count") {
45 | val jobDao = JobDAOMetricsMonitor.newInstance(dao, MetricsLevel.BASIC)
46 | val meters = MetricsWrapper.getRegistry.getMeters
47 | val histograms = MetricsWrapper.getRegistry.getHistograms
48 |
49 | // Counts of meters should be 0
50 | basicMetrics.foreach {
51 | metricName =>
52 | meters.get(metricName).getCount should equal(0)
53 | }
54 |
55 | jobDao.saveJar(appName, DateTime.now(), jarBytes)
56 | jobDao.saveJobInfo(jobInfo)
57 | jobDao.saveJobConfig(jobId, jobConfig)
58 | jobDao.saveContextConfig(contextName, contextConfig)
59 |
60 | // Counts of meters should be incremented
61 | basicMetrics.foreach {
62 | metricName =>
63 | meters.get(metricName).getCount should equal(1)
64 | }
65 |
66 | jobDao.saveJar(appName, DateTime.now(), jarBytes)
67 | jobDao.saveJobInfo(jobInfo)
68 | jobDao.saveJobConfig(jobId, jobConfig)
69 | jobDao.saveContextConfig(contextName, contextConfig)
70 |
71 | // Counts of meters should be incremented again
72 | basicMetrics.foreach {
73 | metricName =>
74 | meters.get(metricName).getCount should equal(2)
75 | }
76 |
77 | // Fine metrics shouldn't be updated
78 | fineMetrics.foreach {
79 | metricName =>
80 | histograms.get(metricName).getCount should equal(0)
81 | }
82 | }
83 |
84 | it("should update fine metrics") {
85 | val jobDao = JobDAOMetricsMonitor.newInstance(dao, MetricsLevel.FINE)
86 | val meters = MetricsWrapper.getRegistry.getMeters
87 | val histograms = MetricsWrapper.getRegistry.getHistograms
88 |
89 | // Fine metrics should be in reset state.
90 | fineMetrics.foreach {
91 | metricName =>
92 | histograms.get(metricName).getCount should equal(0)
93 | }
94 |
95 | jobDao.saveJar(appName, DateTime.now(), jarBytes)
96 | jobDao.saveJobInfo(jobInfo)
97 | jobDao.saveJobConfig(jobId, jobConfig)
98 | jobDao.saveContextConfig(contextName, contextConfig)
99 |
100 | // Fine metrics count should be updated
101 | fineMetrics.foreach {
102 | metricName =>
103 | histograms.get(metricName).getCount should equal(1)
104 | }
105 |
106 | jobDao.saveJar(appName, DateTime.now(), jarBytes)
107 | jobDao.saveJobInfo(jobInfo)
108 | jobDao.saveJobConfig(jobId, jobConfig)
109 | jobDao.saveContextConfig(contextName, contextConfig)
110 |
111 | // Fine metrics count should be updated again
112 | fineMetrics.foreach {
113 | metricName =>
114 | histograms.get(metricName).getCount should equal(2)
115 | }
116 |
117 | // Verify values inside fine metrics
118 | fineMetrics.foreach {
119 | metricName => {
120 | val h = histograms.get(metricName).getSnapshot
121 | if (metricName.endsWith("JarSize")) {
122 | h.getMedian should equal(appName.length + jarBytes.length)
123 | } else if (metricName.endsWith("JobInfoSize")) {
124 | h.getMedian should equal(contextName.length + classPath.length)
125 | } else if (metricName.endsWith("JobConfigSize")) {
126 | val configStr = jobConfig.root().render(ConfigRenderOptions.concise())
127 | h.getMedian should equal(configStr.length)
128 | } else if (metricName.endsWith("ContextConfigSize")) {
129 | val configStr = contextConfig.root().render(ConfigRenderOptions.concise())
130 | h.getMedian should equal(contextName.length + configStr.length)
131 | } else {
132 | fail("Metric " + metricName + " not verified")
133 | }
134 | }
135 | }
136 | }
137 | }
138 | }
139 |
--------------------------------------------------------------------------------
/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/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/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/io/JobFileDAOSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.io
2 |
3 | import java.io.File
4 |
5 | import com.google.common.io.Files
6 | import com.typesafe.config.{Config, ConfigFactory, ConfigValueFactory}
7 | import org.apache.commons.io.FileUtils
8 | import org.joda.time.DateTime
9 | import org.scalatest.matchers.ShouldMatchers
10 | import org.scalatest.{BeforeAndAfter, FunSpec}
11 | import spark.jobserver.TestJarFinder
12 |
13 | class JobFileDAOSpec extends TestJarFinder with FunSpec with ShouldMatchers with BeforeAndAfter {
14 | private val config = ConfigFactory.load("local.test.jobfiledao.conf")
15 |
16 | var dao: JobFileDAO = _
17 |
18 | // *** TEST DATA ***
19 | val time: DateTime = new DateTime()
20 | val throwable: Throwable = new Throwable("test-error")
21 | // jar test data
22 | val jarInfo: JarInfo = genJarInfo(false, false)
23 | val jarBytes: Array[Byte] = Files.toByteArray(testJar)
24 | var jarFile: File = new File(config.getString("spark.jobserver.filedao.rootdir"),
25 | createJarName(jarInfo.appName, jarInfo.uploadTime) + ".jar")
26 |
27 | // jobInfo test data
28 | val jobInfoNoEndNoErr:JobInfo = genJobInfo(jarInfo, false, false, false)
29 | val expectedJobInfo = jobInfoNoEndNoErr
30 | val jobInfoSomeEndNoErr: JobInfo = genJobInfo(jarInfo, true, false, false)
31 | val jobInfoNoEndSomeErr: JobInfo = genJobInfo(jarInfo, false, true, false)
32 | val jobInfoSomeEndSomeErr: JobInfo = genJobInfo(jarInfo, true, true, false)
33 |
34 | // job config test data
35 | val jobId: String = jobInfoNoEndNoErr.jobId
36 | val jobConfig: Config = ConfigFactory.parseString("{marco=pollo}")
37 | val expectedConfig: Config = ConfigFactory.empty().withValue("marco", ConfigValueFactory.fromAnyRef("pollo"))
38 |
39 | private def createJarName(appName: String, uploadTime: DateTime): String = appName + "-" + uploadTime.toString().replace(':', '_')
40 |
41 | // Helper functions and closures!!
42 | private def genJarInfoClosure = {
43 | var appCount: Int = 0
44 | var timeCount: Int = 0
45 |
46 | def genTestJarInfo(newAppName: Boolean, newTime: Boolean): JarInfo = {
47 | appCount = appCount + (if (newAppName) 1 else 0)
48 | timeCount = timeCount + (if (newTime) 1 else 0)
49 |
50 | val app = "test-appName" + appCount
51 | val upload = if (newTime) time.plusMinutes(timeCount) else time
52 |
53 | JarInfo(app, upload)
54 | }
55 |
56 | genTestJarInfo _
57 | }
58 |
59 | private def genJobInfoClosure = {
60 | var count: Int = 0
61 |
62 | def genTestJobInfo(jarInfo: JarInfo, hasEndTime: Boolean, hasError: Boolean, isNew:Boolean): JobInfo = {
63 | count = count + (if (isNew) 1 else 0)
64 |
65 | val id: String = "test-id" + count
66 | val contextName: String = "test-context"
67 | val classPath: String = "test-classpath"
68 | val startTime: DateTime = time
69 |
70 | val noEndTime: Option[DateTime] = None
71 | val someEndTime: Option[DateTime] = Some(time) // Any DateTime Option is fine
72 | val noError: Option[Throwable] = None
73 | val someError: Option[Throwable] = Some(throwable)
74 |
75 | val endTime: Option[DateTime] = if (hasEndTime) someEndTime else noEndTime
76 | val error: Option[Throwable] = if (hasError) someError else noError
77 |
78 | JobInfo(id, contextName, jarInfo, classPath, startTime, endTime, error)
79 | }
80 |
81 | genTestJobInfo _
82 | }
83 |
84 | def genJarInfo = genJarInfoClosure
85 | def genJobInfo = genJobInfoClosure
86 | //**********************************
87 |
88 | before {
89 | FileUtils.deleteDirectory(new File(config.getString("spark.jobserver.filedao.rootdir")))
90 | dao = new JobFileDAO(config)
91 | jarFile.delete()
92 | }
93 |
94 | describe("save and get the jars") {
95 | it("should be able to save one jar and get it back") {
96 | // check the pre-condition
97 | jarFile.exists() should equal (false)
98 |
99 | // save
100 | dao.saveJar(jarInfo.appName, jarInfo.uploadTime, jarBytes)
101 |
102 | // read it back
103 | val apps = dao.getApps
104 |
105 | // test
106 | jarFile.exists() should equal (true)
107 | apps.keySet should equal (Set(jarInfo.appName))
108 | apps(jarInfo.appName) should equal (jarInfo.uploadTime)
109 | }
110 | }
111 |
112 | describe("saveJobConfig() and getJobConfigs() tests") {
113 | it("should provide an empty map on getJobConfigs() for an empty CONFIGS table") {
114 | (Map.empty[String, Config]) should equal (dao.getJobConfigs)
115 | }
116 |
117 | it("should save and get the same config") {
118 | // save job config
119 | dao.saveJobConfig(jobId, jobConfig)
120 |
121 | // get all configs
122 | val configs = dao.getJobConfigs
123 |
124 | // test
125 | configs.keySet should equal (Set(jobId))
126 | configs(jobId) should equal (expectedConfig)
127 | }
128 |
129 | it("should save and get the large config") {
130 | val total = 5000
131 | val str = "{" + (1 to total).map(i => s"key-$i=value-$i").mkString(",") + "}"
132 |
133 | str.getBytes.length > 65535 should equal (true)
134 |
135 | val jobConfig: Config = ConfigFactory.parseString(str)
136 | // save job config
137 | dao.saveJobConfig(jobId, jobConfig)
138 |
139 | // get all configs
140 | val configs = dao.getJobConfigs
141 |
142 | // test
143 | configs.keySet should equal (Set(jobId))
144 | configs(jobId).entrySet().size() should equal (total)
145 | }
146 | }
147 |
148 | describe("Basic saveJobInfo() and getJobInfos() tests") {
149 | it("should provide an empty map on getJobInfos() for an empty JOBS table") {
150 | (Map.empty[String, JobInfo]) should equal (dao.getJobInfos)
151 | }
152 |
153 | it("should save a new JobInfo and get the same JobInfo") {
154 | // save JobInfo
155 | dao.saveJobInfo(jobInfoNoEndNoErr)
156 |
157 | // get all JobInfos
158 | val jobs = dao.getJobInfos
159 |
160 | // test
161 | jobs.keySet should equal (Set(jobId))
162 | jobs(jobId) should equal (expectedJobInfo)
163 | }
164 | }
165 | }
166 |
--------------------------------------------------------------------------------
/job-server/test/spark.jobserver/io/JobCacheDAOSpec.scala:
--------------------------------------------------------------------------------
1 | package spark.jobserver.io
2 |
3 | import java.io.File
4 |
5 | import com.google.common.io.Files
6 | import com.typesafe.config.{Config, ConfigFactory, ConfigValueFactory}
7 | import org.apache.commons.io.FileUtils
8 | import org.joda.time.DateTime
9 | import org.scalatest.matchers.ShouldMatchers
10 | import org.scalatest.{BeforeAndAfter, FunSpec}
11 | import spark.jobserver.TestJarFinder
12 |
13 | class JobCacheDAOSpec extends TestJarFinder with FunSpec with ShouldMatchers with BeforeAndAfter {
14 | private val config = ConfigFactory.load("local.test.jobcachedao.conf")
15 |
16 | var dao: JobCacheDAO = _
17 |
18 | // *** TEST DATA ***
19 | val time: DateTime = new DateTime()
20 | val throwable: Throwable = new Throwable("test-error")
21 | // jar test data
22 | val jarInfo: JarInfo = genJarInfo(false, false)
23 | val jarBytes: Array[Byte] = Files.toByteArray(testJar)
24 | var jarFile: File = new File(config.getString("spark.jobserver.cachedao.rootdir"),
25 | createJarName(jarInfo.appName, jarInfo.uploadTime) + ".jar")
26 |
27 | // jobInfo test data
28 | val jobInfoNoEndNoErr:JobInfo = genJobInfo(jarInfo, false, false, false)
29 | val expectedJobInfo = jobInfoNoEndNoErr
30 | val jobInfoSomeEndNoErr: JobInfo = genJobInfo(jarInfo, true, false, false)
31 | val jobInfoNoEndSomeErr: JobInfo = genJobInfo(jarInfo, false, true, false)
32 | val jobInfoSomeEndSomeErr: JobInfo = genJobInfo(jarInfo, true, true, false)
33 |
34 | // job config test data
35 | val jobId: String = jobInfoNoEndNoErr.jobId
36 | val jobConfig: Config = ConfigFactory.parseString("{marco=pollo}")
37 | val expectedConfig: Config = ConfigFactory.empty().withValue("marco", ConfigValueFactory.fromAnyRef("pollo"))
38 |
39 | private def createJarName(appName: String, uploadTime: DateTime): String = appName + "-" + uploadTime.toString().replace(':', '_')
40 |
41 | // Helper functions and closures!!
42 | private def genJarInfoClosure = {
43 | var appCount: Int = 0
44 | var timeCount: Int = 0
45 |
46 | def genTestJarInfo(newAppName: Boolean, newTime: Boolean): JarInfo = {
47 | appCount = appCount + (if (newAppName) 1 else 0)
48 | timeCount = timeCount + (if (newTime) 1 else 0)
49 |
50 | val app = "test-appName" + appCount
51 | val upload = if (newTime) time.plusMinutes(timeCount) else time
52 |
53 | JarInfo(app, upload)
54 | }
55 |
56 | genTestJarInfo _
57 | }
58 |
59 | private def genJobInfoClosure = {
60 | var count: Int = 0
61 |
62 | def genTestJobInfo(jarInfo: JarInfo, hasEndTime: Boolean, hasError: Boolean, isNew:Boolean): JobInfo = {
63 | count = count + (if (isNew) 1 else 0)
64 |
65 | val id: String = "test-id" + count
66 | val contextName: String = "test-context"
67 | val classPath: String = "test-classpath"
68 | val startTime: DateTime = time
69 |
70 | val noEndTime: Option[DateTime] = None
71 | val someEndTime: Option[DateTime] = Some(time) // Any DateTime Option is fine
72 | val noError: Option[Throwable] = None
73 | val someError: Option[Throwable] = Some(throwable)
74 |
75 | val endTime: Option[DateTime] = if (hasEndTime) someEndTime else noEndTime
76 | val error: Option[Throwable] = if (hasError) someError else noError
77 |
78 | JobInfo(id, contextName, jarInfo, classPath, startTime, endTime, error)
79 | }
80 |
81 | genTestJobInfo _
82 | }
83 |
84 | def genJarInfo = genJarInfoClosure
85 | def genJobInfo = genJobInfoClosure
86 | //**********************************
87 |
88 | before {
89 | FileUtils.deleteDirectory(new File(config.getString("spark.jobserver.cachedao.rootdir")))
90 | dao = new JobCacheDAO(config)
91 | jarFile.delete()
92 | }
93 |
94 | describe("save and get the jars") {
95 | it("should be able to save one jar and get it back") {
96 | // check the pre-condition
97 | jarFile.exists() should equal (false)
98 |
99 | // save
100 | dao.saveJar(jarInfo.appName, jarInfo.uploadTime, jarBytes)
101 |
102 | // read it back
103 | val apps = dao.getApps
104 |
105 | // test
106 | jarFile.exists() should equal (true)
107 | apps.keySet should equal (Set(jarInfo.appName))
108 | apps(jarInfo.appName) should equal (jarInfo.uploadTime)
109 | }
110 | }
111 |
112 | describe("saveJobConfig() and getJobConfigs() tests") {
113 | it("should provide an empty map on getJobConfigs() for an empty CONFIGS table") {
114 | (Map.empty[String, Config]) should equal (dao.getJobConfigs)
115 | }
116 |
117 | it("should save and get the same config") {
118 | // save job config
119 | dao.saveJobConfig(jobId, jobConfig)
120 |
121 | // get all configs
122 | val configs = dao.getJobConfigs
123 |
124 | // test
125 | configs.keySet should equal (Set(jobId))
126 | configs(jobId) should equal (expectedConfig)
127 | }
128 |
129 | it("should save and get the large config") {
130 | val total = 5000
131 | val str = "{" + (1 to total).map(i => s"key-$i=value-$i").mkString(",") + "}"
132 |
133 | str.getBytes.length > 65535 should equal (true)
134 |
135 | val jobConfig: Config = ConfigFactory.parseString(str)
136 | // save job config
137 | dao.saveJobConfig(jobId, jobConfig)
138 |
139 | // get all configs
140 | val configs = dao.getJobConfigs
141 |
142 | // test
143 | configs.keySet should equal (Set(jobId))
144 | configs(jobId).entrySet().size() should equal (total)
145 | }
146 | }
147 |
148 | describe("Basic saveJobInfo() and getJobInfos() tests") {
149 | it("should provide an empty map on getJobInfos() for an empty JOBS table") {
150 | (Map.empty[String, JobInfo]) should equal (dao.getJobInfos)
151 | }
152 |
153 | it("should save a new JobInfo and get the same JobInfo") {
154 | // save JobInfo
155 | dao.saveJobInfo(jobInfoNoEndNoErr)
156 |
157 | // get all JobInfos
158 | val jobs = dao.getJobInfos
159 |
160 | // test
161 | jobs.keySet should equal (Set(jobId))
162 | jobs(jobId) should equal (expectedJobInfo)
163 | }
164 | }
165 | }
166 |
--------------------------------------------------------------------------------