├── .gitignore ├── LICENSE.md ├── README.md ├── akka-app ├── README.md ├── src │ └── ooyala.common.akka │ │ ├── ActorMetrics.scala │ │ ├── ActorStack.scala │ │ ├── AkkaTestUtils.scala │ │ ├── InstrumentedActor.scala │ │ ├── Slf4jLogging.scala │ │ ├── actor │ │ └── Reaper.scala │ │ ├── metrics │ │ ├── CompactMetricsWriter.scala │ │ ├── DatadogConfigParser.scala │ │ ├── JvmMetricsWrapper.scala │ │ ├── MetricsLevel.scala │ │ ├── MetricsWrapper.scala │ │ └── YammerMetrics.scala │ │ └── web │ │ ├── CommonRoutes.scala │ │ ├── JsonUtils.scala │ │ └── WebService.scala └── test │ └── ooyala.common.akka │ ├── ActorMetricsSpec.scala │ ├── ActorStackSpec.scala │ ├── actor │ └── ReaperSpec.scala │ ├── metrics │ ├── DatadogConfigParserSpec.scala │ ├── JvmMetricsWrapperSpec.scala │ └── MetricsLevelSpec.scala │ └── web │ ├── CommonRoutesSpec.scala │ └── JsonUtilsSpec.scala ├── bin ├── server_deploy.sh ├── server_host_setup.sh ├── server_package.sh ├── server_start.sh └── server_stop.sh ├── config ├── doc ├── actors.md ├── job-server-flow.md ├── start-job-msg-flow-AdHocJobManagerActor.md └── success-job-flow.md ├── job-server-api └── src │ └── spark.jobserver │ ├── NamedRddSupport.scala │ └── SparkJob.scala ├── job-server-tests └── src │ └── spark.jobserver │ ├── LongPiJob.scala │ ├── NoOpJob.scala │ ├── SparkTestJobs.scala │ ├── VeryShortDoubleJob.scala │ └── WordCountExample.scala ├── job-server ├── config │ ├── local.conf.template │ ├── local.sh.template │ ├── log4j-local.properties │ ├── log4j-server.properties │ ├── logback-local.xml │ └── logback-server.xml ├── src │ ├── main │ │ └── resources │ │ │ ├── application.conf │ │ │ └── html │ │ │ ├── bootstrap │ │ │ ├── css │ │ │ │ └── bootstrap.min.css │ │ │ └── js │ │ │ │ └── bootstrap.min.js │ │ │ ├── css │ │ │ └── index.css │ │ │ ├── index.html │ │ │ └── js │ │ │ └── spark-jobserver-ui.js │ ├── spark.jobserver │ │ ├── CommonMessages.scala │ │ ├── JarManager.scala │ │ ├── JarUtils.scala │ │ ├── JobCache.scala │ │ ├── JobInfoActor.scala │ │ ├── JobManagerActor.scala │ │ ├── JobResultActor.scala │ │ ├── JobServer.scala │ │ ├── JobServerNamedRdds.scala │ │ ├── JobStatusActor.scala │ │ ├── LocalContextSupervisorActor.scala │ │ ├── RddManagerActor.scala │ │ ├── SparkWebUiActor.scala │ │ ├── WebApi.scala │ │ ├── io │ │ │ ├── JobCacheDAO.scala │ │ │ ├── JobDAO.scala │ │ │ ├── JobDAOMetricsMonitor.scala │ │ │ ├── JobFileDAO.scala │ │ │ ├── JobSqlDAO.scala │ │ │ └── JobSqlDAOJdbcConfig.scala │ │ └── util │ │ │ ├── ContextURLClassLoader.scala │ │ │ ├── DateUtils.scala │ │ │ ├── GuavaCacheUtils.scala │ │ │ ├── LRUCache.scala │ │ │ ├── SparkContextFactory.scala │ │ │ └── SparkJobUtils.scala │ └── test │ │ └── resources │ │ ├── local.test.jobcachedao.conf │ │ ├── local.test.jobfiledao.conf │ │ ├── local.test.jobsqldao.conf │ │ ├── log4j.properties │ │ └── logback-test.xml └── test │ └── spark.jobserver │ ├── InMemoryDAO.scala │ ├── JobInfoActorSpec.scala │ ├── JobManagerActorAdHocSpec.scala │ ├── JobManagerActorSpec.scala │ ├── JobManagerSpec.scala │ ├── JobResultActorSpec.scala │ ├── JobResultCacheSpec.scala │ ├── JobStatusActorSpec.scala │ ├── LocalContextSupervisorHaSpec.scala │ ├── LocalContextSupervisorSpec.scala │ ├── NamedRddsSpec.scala │ ├── SparkJobSpec.scala │ ├── SparkWebUiActorSpec.scala │ ├── TestJarFinder.scala │ ├── WebApiSpec.scala │ ├── io │ ├── JobCacheDAOSpec.scala │ ├── JobDAOMetricsMonitorSpec.scala │ ├── JobFileDAOSpec.scala │ ├── JobSqlDAOJdbcConfigSpec.scala │ └── JobSqlDAOSpec.scala │ ├── stress │ └── SingleContextJobStress.scala │ └── util │ └── SparkJobUtilsSpec.scala ├── project ├── Assembly.scala ├── Build.scala ├── Dependencies.scala ├── build.properties └── plugins.sbt └── scalastyle-config.xml /.gitignore: -------------------------------------------------------------------------------- 1 | # Standard sbt ignores 2 | target/ 3 | src_managed/ 4 | project/boot/ 5 | test-reports/ 6 | *.log* 7 | .ensime* 8 | .idea* 9 | .vagrant 10 | 11 | # ignore deployment configs 12 | config/*.conf 13 | config/*.sh -------------------------------------------------------------------------------- /LICENSE.md: -------------------------------------------------------------------------------- 1 | Copyright 2014 Ooyala, Inc. 2 | 3 | Licensed under the Apache License, Version 2.0 (the "License"); 4 | you may not use this file except in compliance with the License. 5 | You may obtain a copy of the License at 6 | 7 | http://www.apache.org/licenses/LICENSE-2.0 8 | 9 | Unless required by applicable law or agreed to in writing, software 10 | distributed under the License is distributed on an "AS IS" BASIS, 11 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | See the License for the specific language governing permissions and 13 | limitations under the License. -------------------------------------------------------------------------------- /akka-app/README.md: -------------------------------------------------------------------------------- 1 | akka_app defines a standard stack for Akka applications, as well as traits and utilities for Akka Actors. 2 | 3 | ## The included Akka Stack 4 | 5 | * Akka 2.2.4 (including remote + testkit) 6 | * JodaTime 7 | * Yammer Metrics for stats and instrumentation 8 | * spray-json for JSON serialization 9 | * spray for embedded web server, with some common routes like /metricz and /statusz 10 | 11 | ## Useful Traits and Utilities 12 | 13 | * ActorStack - a base trait for enabling stackable Akka Actor traits 14 | * Slf4jLogging - directly log to Slf4j + add akkaSource MDC context for the actor's path 15 | * ActorMetrics - instrument receive handler duration and invocation frequency 16 | * CommonRoutes - /metricz, /statusz 17 | 18 | ## For more info 19 | 20 | Please see the following presentation for more info on how we use stackable traits and Akka at Ooyala: 21 | 22 | http://www.slideshare.net/EvanChan2/akka-inproductionpnw-scala2013 23 | -------------------------------------------------------------------------------- /akka-app/src/ooyala.common.akka/ActorMetrics.scala: -------------------------------------------------------------------------------- 1 | package ooyala.common.akka 2 | 3 | import ooyala.common.akka.metrics.MetricsWrapper 4 | 5 | /** 6 | * ActorMetrics is a trait that provides the following metrics: 7 | * * message-handler.meter.{mean,m1,m5,m15} = moving avg of rate at which receive handler is called 8 | * * message-handler.duration.{mean,p75,p99,p999} = histogram of wrappedReeive() running time 9 | * 10 | * NOTE: the number of incoming messages can be tracked using meter.count. 11 | */ 12 | trait ActorMetrics extends ActorStack { 13 | // Timer includes a histogram of wrappedReceive() duration as well as moving avg of rate of invocation 14 | val metricReceiveTimer = MetricsWrapper.newTimer(getClass, "message-handler") 15 | 16 | override def receive: Receive = { 17 | case x => 18 | val context = metricReceiveTimer.time() 19 | try { 20 | super.receive(x) 21 | } finally { 22 | context.stop() 23 | } 24 | } 25 | } -------------------------------------------------------------------------------- /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/AkkaTestUtils.scala: -------------------------------------------------------------------------------- 1 | package ooyala.common.akka 2 | 3 | import akka.actor.{ActorSystem, ActorRef} 4 | import akka.pattern.gracefulStop 5 | import scala.concurrent.Await 6 | 7 | object AkkaTestUtils { 8 | import scala.concurrent.duration._ 9 | 10 | // This is a var for now because we need to let people change it, and we can't pass this in as a param 11 | // because then we would change the API. If we have it as a default param, we can't have multiple methods 12 | // with the same name. 13 | var timeout = 10 seconds 14 | 15 | def shutdownAndWait(actor: ActorRef) { 16 | if (actor != null) { 17 | val stopped = gracefulStop(actor, timeout) 18 | Await.result(stopped, timeout + (1 seconds)) 19 | } 20 | } 21 | 22 | def shutdownAndWait(system: ActorSystem) { 23 | if (system != null) { 24 | system.shutdown() 25 | system.awaitTermination(timeout) 26 | } 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /akka-app/src/ooyala.common.akka/InstrumentedActor.scala: -------------------------------------------------------------------------------- 1 | package ooyala.common.akka 2 | 3 | import akka.actor.Actor 4 | 5 | /** 6 | * Base class that includes Slf4jLogging and ActorMetrics for convenience 7 | */ 8 | abstract class InstrumentedActor extends Actor with Slf4jLogging with ActorMetrics { 9 | /** preRestart() is called when actor is killed due to exception, and will be restarted. It is 10 | * run on the current actor instance that is about to be killed. We just log errors. 11 | * The super (original) method should call postStop() and shut down children as well. 12 | */ 13 | override def preRestart(reason: Throwable, message: Option[Any]) { 14 | logger.error("About to restart actor due to exception:", reason) 15 | super.preRestart(reason, message) 16 | } 17 | 18 | /** postStop() is called when actor is stopped or restarted due to Exceptions **/ 19 | override def postStop() { logger.warn("Shutting down {}", getClass.getName) } 20 | } 21 | -------------------------------------------------------------------------------- /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 | -------------------------------------------------------------------------------- /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 | -------------------------------------------------------------------------------- /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/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/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 | -------------------------------------------------------------------------------- /akka-app/src/ooyala.common.akka/metrics/MetricsLevel.scala: -------------------------------------------------------------------------------- 1 | package ooyala.common.akka.metrics 2 | 3 | sealed abstract class MetricsLevel( 4 | val level: Int, 5 | val name: String) 6 | extends Ordered[MetricsLevel] { 7 | 8 | def compare(that: MetricsLevel) = level - that.level 9 | 10 | override def toString = name 11 | } 12 | 13 | /** 14 | * Specifies the different levels of details on metrics. 15 | */ 16 | object MetricsLevel { 17 | case object NONE extends MetricsLevel(0, "NONE") 18 | case object BASIC extends MetricsLevel(1, "BASIC") 19 | case object FINE extends MetricsLevel(2, "FINE") 20 | case object FINER extends MetricsLevel(3, "FINER") 21 | 22 | /** 23 | * Converts an integer value to a MetricsLevel 24 | * 25 | * @param level an integer value 26 | * @return a MetricsLevel for the integer value 27 | */ 28 | def valueOf(level: Int): MetricsLevel = level match { 29 | case 0 => NONE 30 | case 1 => BASIC 31 | case 2 => FINE 32 | case 3 => FINER 33 | case _ => { 34 | throw new IllegalArgumentException("Metrics level must be 0 to 3") 35 | } 36 | } 37 | } -------------------------------------------------------------------------------- /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 | } -------------------------------------------------------------------------------- /akka-app/src/ooyala.common.akka/metrics/YammerMetrics.scala: -------------------------------------------------------------------------------- 1 | package ooyala.common.akka.metrics 2 | 3 | import com.yammer.metrics.Metrics 4 | import com.yammer.metrics.core.{Histogram, Meter, Gauge} 5 | import java.util.concurrent.TimeUnit 6 | 7 | /** 8 | * Utility trait to make metrics creation slightly less verbose 9 | */ 10 | trait YammerMetrics { 11 | def meter(name: String, eventType: String): Meter = 12 | Metrics.newMeter(getClass, name, eventType, TimeUnit.SECONDS) 13 | 14 | def gauge[T](name: String, metric: => T, scope: String = null): Gauge[T] = 15 | Metrics.newGauge(getClass, name, scope, new Gauge[T] { 16 | override def value(): T = metric 17 | }) 18 | 19 | def histogram(name: String): Histogram = Metrics.newHistogram(getClass, name, true) 20 | } 21 | -------------------------------------------------------------------------------- /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 | -------------------------------------------------------------------------------- /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/web/WebService.scala: -------------------------------------------------------------------------------- 1 | package ooyala.common.akka.web 2 | 3 | import akka.actor.ActorSystem 4 | import spray.routing.{Route, SimpleRoutingApp} 5 | 6 | /** 7 | * Contains methods for starting an embedded Spray web server. 8 | */ 9 | object WebService extends SimpleRoutingApp { 10 | /** 11 | * Starts a web server given a Route. Note that this call is meant to be made from an App or other top 12 | * level scope, and not within an actor, as system.actorOf may block. 13 | * 14 | * @param route The spray Route for the service. Multiple routes can be combined like (route1 ~ route2). 15 | * @param system the ActorSystem to use 16 | * @param host The host string to bind to, defaults to "0.0.0.0" 17 | * @param port The port number to bind to 18 | */ 19 | def start(route: Route, system: ActorSystem, 20 | host: String = "0.0.0.0", port: Int = 8080) { 21 | implicit val actorSystem = system 22 | startServer(host, port)(route) 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /akka-app/test/ooyala.common.akka/ActorMetricsSpec.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 ActorMetricsSpec extends FunSpec with ShouldMatchers { 11 | implicit val system = ActorSystem("test") 12 | 13 | describe("actor metrics") { 14 | it("should increment receive count metric when a message is received") { 15 | val actorRef = TestActorRef(new DummyActor with ActorMetrics) 16 | val actor = actorRef.underlyingActor 17 | 18 | actorRef ! "me" 19 | actor.metricReceiveTimer.getCount should equal (1) 20 | } 21 | } 22 | } -------------------------------------------------------------------------------- /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 | } -------------------------------------------------------------------------------- /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 | -------------------------------------------------------------------------------- /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/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 | -------------------------------------------------------------------------------- /akka-app/test/ooyala.common.akka/metrics/MetricsLevelSpec.scala: -------------------------------------------------------------------------------- 1 | package ooyala.common.akka.metrics 2 | 3 | import org.scalatest.FunSpec 4 | import org.scalatest.matchers.ShouldMatchers 5 | 6 | class MetricsLevelSpec extends FunSpec with ShouldMatchers { 7 | 8 | describe("MetricsLevel") { 9 | it("should return a valid metric level") { 10 | MetricsLevel.valueOf(0) should equal(MetricsLevel.NONE) 11 | 12 | MetricsLevel.valueOf(1) should equal(MetricsLevel.BASIC) 13 | 14 | MetricsLevel.valueOf(2) should equal(MetricsLevel.FINE) 15 | 16 | MetricsLevel.valueOf(3) should equal(MetricsLevel.FINER) 17 | } 18 | 19 | it("should throw exceptions") { 20 | evaluating { 21 | MetricsLevel.valueOf(-1) 22 | } should produce [IllegalArgumentException] 23 | 24 | evaluating { 25 | MetricsLevel.valueOf(4) 26 | } should produce [IllegalArgumentException] 27 | } 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /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 | } -------------------------------------------------------------------------------- /bin/server_deploy.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # Script for deploying the job server to a host 3 | ENV=$1 4 | if [ -z "$ENV" ]; then 5 | echo "Syntax: $0 " 6 | echo " for a list of environments, ls config/*.sh" 7 | exit 0 8 | fi 9 | 10 | if [ -z "$CONFIG_DIR" ]; then 11 | CONFIG_DIR="$(dirname $0)/config" 12 | fi 13 | configFile="$CONFIG_DIR/$ENV.sh" 14 | if [ ! -f "$configFile" ]; then 15 | echo "Could not find $configFile" 16 | exit 1 17 | fi 18 | . $configFile 19 | 20 | echo Deploying job server to $DEPLOY_HOSTS... 21 | 22 | cd $(dirname $0)/.. 23 | sbt job-server/assembly 24 | if [ "$?" != "0" ]; then 25 | echo "Assembly failed" 26 | exit 1 27 | fi 28 | 29 | FILES="job-server/target/spark-job-server.jar 30 | bin/server_start.sh 31 | bin/server_stop.sh 32 | $CONFIG_DIR/$ENV.conf 33 | config/log4j-server.properties" 34 | 35 | for host in $DEPLOY_HOSTS; do 36 | # We assume that the deploy user is APP_USER and has permissions 37 | ssh ${APP_USER}@$host mkdir -p $INSTALL_DIR 38 | scp $FILES ${APP_USER}@$host:$INSTALL_DIR/ 39 | scp $configFile ${APP_USER}@$host:$INSTALL_DIR/settings.sh 40 | done 41 | -------------------------------------------------------------------------------- /bin/server_host_setup.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # Script for setting up folders for job server deployment 3 | ENV=$1 4 | if [ -z "$ENV" ]; then 5 | echo "Syntax: $0 " 6 | echo " for a list of environments, ls config/*.sh" 7 | exit 0 8 | fi 9 | 10 | if [ -z "$CONFIG_DIR" ]; then 11 | CONFIG_DIR="$(dirname $0)/config" 12 | fi 13 | configFile="$CONFIG_DIR/$ENV.sh" 14 | if [ ! -f "$configFile" ]; then 15 | echo "Could not find $configFile" 16 | exit 1 17 | fi 18 | . $configFile 19 | 20 | echo Setting up ${DEPLOY_HOSTS}... 21 | 22 | for host in $DEPLOY_HOSTS; do 23 | ssh root@${host} mkdir -p $INSTALL_DIR 24 | ssh root@${host} chown ${APP_USER}:${APP_GROUP} $INSTALL_DIR 25 | ssh root@${host} mkdir -p $LOG_DIR 26 | ssh root@${host} chown ${APP_USER}:${APP_GROUP} $LOG_DIR 27 | done 28 | -------------------------------------------------------------------------------- /bin/server_package.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # Script for packaging all the job server files to .tar.gz for Mesos or other single-image deploys 3 | WORK_DIR=/tmp/job-server 4 | 5 | ENV=$1 6 | if [ -z "$ENV" ]; then 7 | echo "Syntax: $0 " 8 | echo " for a list of environments, ls config/*.sh" 9 | exit 0 10 | fi 11 | 12 | if [ -z "$CONFIG_DIR" ]; then 13 | CONFIG_DIR="$(dirname $0)/config" 14 | fi 15 | configFile="$CONFIG_DIR/$ENV.sh" 16 | 17 | echo Packaging job-server for environment $ENV... 18 | 19 | cd $(dirname $0)/.. 20 | sbt job-server/assembly 21 | if [ "$?" != "0" ]; then 22 | echo "Assembly failed" 23 | exit 1 24 | fi 25 | 26 | FILES="job-server/target/spark-job-server.jar 27 | bin/server_start.sh 28 | bin/server_stop.sh 29 | $CONFIG_DIR/$ENV.conf 30 | config/log4j-server.properties" 31 | 32 | rm -rf $WORK_DIR 33 | mkdir -p $WORK_DIR 34 | cp $FILES $WORK_DIR/ 35 | cp $configFile $WORK_DIR/settings.sh 36 | pushd $WORK_DIR 37 | TAR_FILE=$WORK_DIR/job-server.tar.gz 38 | rm -f $TAR_FILE 39 | tar zcvf $TAR_FILE * 40 | popd 41 | 42 | echo "Created distribution at $TAR_FILE" 43 | -------------------------------------------------------------------------------- /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 | -------------------------------------------------------------------------------- /bin/server_stop.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # Script to stop the job server 3 | 4 | get_abs_script_path() { 5 | pushd . >/dev/null 6 | cd $(dirname $0) 7 | appdir=$(pwd) 8 | popd >/dev/null 9 | } 10 | 11 | get_abs_script_path 12 | 13 | if [ -f "$appdir/settings.sh" ]; then 14 | . $appdir/settings.sh 15 | else 16 | echo "Missing $appdir/settings.sh, exiting" 17 | exit 1 18 | fi 19 | 20 | if [ ! -f "$PIDFILE" ] || ! kill -0 $(cat "$PIDFILE"); then 21 | echo 'Job server not running' 22 | else 23 | echo 'Stopping job server...' 24 | kill -15 $(cat "$PIDFILE") && rm -f "$PIDFILE" 25 | echo '...job server stopped' 26 | fi 27 | -------------------------------------------------------------------------------- /config: -------------------------------------------------------------------------------- 1 | job-server/config -------------------------------------------------------------------------------- /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 | -------------------------------------------------------------------------------- /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 | -------------------------------------------------------------------------------- /doc/success-job-flow.md: -------------------------------------------------------------------------------- 1 | For full job server flow, see job-server-flow.md. 2 | 3 | user->WebApi: POST /jobs/ 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-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-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 | -------------------------------------------------------------------------------- /job-server-tests/src/spark.jobserver/NoOpJob.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import com.typesafe.config.{Config, ConfigFactory} 4 | import org.apache.spark._ 5 | 6 | /** 7 | * An empty no computation job for stress tests purpose. 8 | */ 9 | object NoOpJob extends SparkJob { 10 | def main(args: Array[String]) { 11 | val sc = new SparkContext("local[4]", "NoOpJob") 12 | val config = ConfigFactory.parseString("") 13 | val results = runJob(sc, config) 14 | println("Result is " + results) 15 | } 16 | 17 | def validate(sc: SparkContext, config: Config): SparkJobValidation = SparkJobValid 18 | 19 | def runJob(sc: SparkContext, config: Config): Any = 1 20 | } 21 | -------------------------------------------------------------------------------- /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/VeryShortDoubleJob.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import com.typesafe.config.{Config, ConfigFactory} 4 | import org.apache.spark._ 5 | 6 | /** 7 | * A very short job for stress tests purpose. 8 | * Small data. Double every value in the data. 9 | */ 10 | object VeryShortDoubleJob extends SparkJob { 11 | private val data = Array(1, 2, 3) 12 | 13 | def main(args: Array[String]) { 14 | val sc = new SparkContext("local[4]", "VeryShortDoubleJob") 15 | val config = ConfigFactory.parseString("") 16 | val results = runJob(sc, config) 17 | println("Result is " + results) 18 | } 19 | 20 | override def validate(sc: SparkContext, config: Config): SparkJobValidation = { 21 | SparkJobValid 22 | } 23 | 24 | override def runJob(sc: SparkContext, config: Config): Any = { 25 | val dd = sc.parallelize(data) 26 | dd.map( _ * 2 ).collect() 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /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 | -------------------------------------------------------------------------------- /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 | -------------------------------------------------------------------------------- /job-server/config/local.sh.template: -------------------------------------------------------------------------------- 1 | # Environment and deploy file 2 | # For use with bin/server_deploy, bin/server_package etc. 3 | DEPLOY_HOSTS="hostname1.net 4 | hostname2.net" 5 | 6 | APP_USER=spark 7 | APP_GROUP=spark 8 | INSTALL_DIR=/home/spark/job-server 9 | LOG_DIR=/var/log/job-server 10 | PIDFILE=spark-jobserver.pid 11 | SPARK_HOME=/home/spark/spark-0.8.0 12 | SPARK_CONF_HOME=$SPARK_HOME/conf 13 | # Only needed for Mesos deploys 14 | SPARK_EXECUTOR_URI=/home/spark/spark-0.8.0.tar.gz 15 | -------------------------------------------------------------------------------- /job-server/config/log4j-local.properties: -------------------------------------------------------------------------------- 1 | # Default LOG4J properties logs to a file and overwrites it every time. Works well for tests. 2 | 3 | # Root logger option 4 | log4j.rootLogger=INFO, LOGFILE 5 | 6 | log4j.appender.LOGFILE=org.apache.log4j.FileAppender 7 | log4j.appender.LOGFILE.File=job-server-local.log 8 | log4j.appender.LOGFILE.layout=org.apache.log4j.PatternLayout 9 | # log4j.appender.LOGFILE.layout.ConversionPattern=%d %-5p %c - %m%n 10 | log4j.appender.LOGFILE.layout.ConversionPattern=[%d] %-5p %.26c [%X{testName}] [%X{akkaSource}] - %m%n 11 | log4j.appender.LOGFILE.Append=false 12 | -------------------------------------------------------------------------------- /job-server/config/log4j-server.properties: -------------------------------------------------------------------------------- 1 | # Rotating log file configuration for server deploys 2 | 3 | # Root logger option 4 | log4j.rootLogger=INFO,LOGFILE 5 | 6 | log4j.appender.LOGFILE=org.apache.log4j.RollingFileAppender 7 | log4j.appender.LOGFILE.File=${LOG_DIR}/spark-job-server.log 8 | log4j.appender.LOGFILE.DatePattern='.'yyyy-MM-dd 9 | log4j.appender.LOGFILE.layout=org.apache.log4j.PatternLayout 10 | # log4j.appender.LOGFILE.layout.ConversionPattern=%d %-5p %c - %m%n 11 | log4j.appender.LOGFILE.layout.ConversionPattern=[%d] %-5p %.26c [%X{testName}] [%X{akkaSource}] - %m%n 12 | log4j.appender.LOGFILE.maxFileSize=20MB 13 | log4j.appender.LOGFILE.maxBackupIndex=30 14 | -------------------------------------------------------------------------------- /job-server/config/logback-local.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | spark-jobserver.log 8 | 9 | 10 | false 11 | 12 | 13 | [%date{ISO8601}] %-5level %logger{26} [%X{jobId}] [%X{akkaSource}] - %msg%n 14 | 15 | 16 | 17 | 18 | 19 | 20 | 21 | 22 | -------------------------------------------------------------------------------- /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/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/css/index.css: -------------------------------------------------------------------------------- 1 | .page-header { 2 | text-align: center; 3 | } 4 | 5 | .table { 6 | width: 98%; 7 | } -------------------------------------------------------------------------------- /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 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | 41 |
IdClasspathContextStart TimeDuration
42 |
43 |
44 |
45 |
Completed Jobs
46 |
47 | 48 | 49 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | 59 |
IdClasspathContextStart TimeDuration
60 |
61 |
62 |
63 |
Failed Jobs
64 |
65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 | 73 | 74 | 75 | 76 | 77 |
IdClasspathContextStart TimeDuration
78 |
79 |
80 |
81 |
82 |
83 |
84 |
85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | 93 |
Name
94 |
95 |
96 |
97 |
98 |
99 |
100 |
101 | 102 | 103 | 104 | 105 | 106 | 107 | 108 | 109 | 110 |
NameDeployment Time
111 |
112 |
113 |
114 |
115 | 116 | 117 | -------------------------------------------------------------------------------- /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/CommonMessages.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.ActorRef 4 | import org.joda.time.DateTime 5 | 6 | trait StatusMessage { 7 | val jobId: String 8 | } 9 | 10 | // Messages that are sent and received by multiple actors. 11 | object CommonMessages { 12 | // job status messages 13 | case class JobStarted(jobId: String, context: String, startTime: DateTime) extends StatusMessage 14 | case class JobFinished(jobId: String, endTime: DateTime) extends StatusMessage 15 | case class JobValidationFailed(jobId: String, endTime: DateTime, err: Throwable) extends StatusMessage 16 | case class JobErroredOut(jobId: String, endTime: DateTime, err: Throwable) extends StatusMessage 17 | 18 | /** 19 | * NOTE: For Subscribe, make sure to use `classOf[]` to get the Class for the case classes above. 20 | * Otherwise, `.getClass` will get the `java.lang.Class` of the companion object. 21 | */ 22 | case class GetJobResult(jobId: String) 23 | case class JobResult(jobId: String, result: Any) 24 | 25 | case class Subscribe(jobId: String, receiver: ActorRef, events: Set[Class[_]]) { 26 | require(events.nonEmpty, "Must subscribe to at least one type of event!") 27 | } 28 | case class Unsubscribe(jobId: String, receiver: ActorRef) // all events for this jobId and receiving actor 29 | 30 | // errors 31 | case object NoSuchJobId 32 | case object NoSuchApplication 33 | case object NoSuchClass 34 | case object JobInitAlready 35 | case class NoJobSlotsAvailable(maxJobSlots: Int) // TODO: maybe rename this 36 | } 37 | -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/JarManager.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import ooyala.common.akka.InstrumentedActor 4 | import spark.jobserver.io.JobDAO 5 | import org.joda.time.DateTime 6 | 7 | // Messages to JarManager actor 8 | case class StoreJar(appName: String, jarBytes: Array[Byte]) 9 | case object ListJars 10 | 11 | // Responses 12 | case object InvalidJar 13 | case object JarStored 14 | 15 | /** 16 | * An Actor that manages the jars stored by the job server. It's important that threads do not try to 17 | * load a class from a jar as a new one is replacing it, so using an actor to serialize requests is perfect. 18 | */ 19 | class JarManager(jobDao: JobDAO) extends InstrumentedActor { 20 | override def wrappedReceive: Receive = { 21 | case ListJars => sender ! createJarsList() 22 | 23 | case StoreJar(appName, jarBytes) => 24 | logger.info("Storing jar for app {}, {} bytes", appName, jarBytes.size) 25 | if (!JarUtils.validateJarBytes(jarBytes)) { 26 | sender ! InvalidJar 27 | } else { 28 | val uploadTime = DateTime.now() 29 | jobDao.saveJar(appName, uploadTime, jarBytes) 30 | sender ! JarStored 31 | } 32 | } 33 | 34 | private def createJarsList() = jobDao.getApps 35 | } 36 | -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/JarUtils.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import java.io.File 4 | import java.lang.ClassLoader 5 | import java.lang.reflect.Constructor 6 | import java.net.{ URL, URLClassLoader } 7 | import org.slf4j.LoggerFactory 8 | 9 | /** 10 | * A set of utilities for dynamically loading classes from a jar file, and saving the jar file. 11 | */ 12 | object JarUtils { 13 | val logger = LoggerFactory.getLogger(getClass) 14 | 15 | /** 16 | * Loads a Scala object or class from a classloader. 17 | * See http://stackoverflow.com/questions/3216780/problem-reloading-a-jar-using-urlclassloader?lq=1 18 | * See http://stackoverflow.com/questions/8867766/scala-dynamic-object-class-loading 19 | * 20 | * 21 | * @param classOrObjectName must be the fully qualified name of the Scala object or class that 22 | * implements the SparkJob trait. If an object is used, do not include the 23 | * trailing '$'. 24 | * @param loader the ClassLoader to use to load the class or object. Typically a URLClassLoader. 25 | * @return Function0[C] to obtain the object/class. Calling the function will return a reference to 26 | * the object (for objects), or a new instance of a class (for classes) that implement the 27 | * SparkJob trait. 28 | */ 29 | def loadClassOrObject[C](classOrObjectName: String, loader: ClassLoader): () => C = { 30 | def fallBackToClass(): () => C = { 31 | val constructor = loadConstructor[C](classOrObjectName, loader) 32 | () => constructor.newInstance() 33 | } 34 | 35 | // Try loading it as an object first, if that fails, then try it as a class 36 | try { 37 | val objectRef = loadObject[C](classOrObjectName + "$", loader) 38 | () => objectRef 39 | } catch { 40 | case e: java.lang.ClassNotFoundException => fallBackToClass() 41 | case e: java.lang.ClassCastException => fallBackToClass() 42 | case e: java.lang.NoSuchMethodException => fallBackToClass() 43 | case e: java.lang.NoSuchFieldException => fallBackToClass() 44 | } 45 | } 46 | 47 | private def loadConstructor[C](className: String, loader: ClassLoader): Constructor[C] = { 48 | logger.info("Loading class {} using loader {}", className: Any, loader) 49 | val loadedClass = loader.loadClass(className).asInstanceOf[Class[C]] 50 | val result = loadedClass.getConstructor() 51 | if (loadedClass.getClassLoader != loader) { 52 | logger.error("Wrong ClassLoader for class {}: Expected {} but got {}", loadedClass.getName, 53 | loader.toString, loadedClass.getClassLoader) 54 | } 55 | result 56 | } 57 | 58 | private def loadObject[C](objectName: String, loader: ClassLoader): C = { 59 | logger.info("Loading object {} using loader {}", objectName: Any, loader) 60 | val loadedClass = loader.loadClass(objectName) 61 | val objectRef = loadedClass.getField("MODULE$").get(null).asInstanceOf[C] 62 | if (objectRef.getClass.getClassLoader != loader) { 63 | logger.error("Wrong ClassLoader for object {}: Expected {} but got {}", objectRef.getClass.getName, 64 | loader.toString, objectRef.getClass.getClassLoader) 65 | } 66 | objectRef 67 | } 68 | 69 | def validateJarBytes(jarBytes: Array[Byte]): Boolean = { 70 | jarBytes.size > 4 && 71 | // For now just check the first few bytes are the ZIP signature: 0x04034b50 little endian 72 | jarBytes(0) == 0x50 && jarBytes(1) == 0x4b && jarBytes(2) == 0x03 && jarBytes(3) == 0x04 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/JobCache.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import java.net.URL 4 | import ooyala.common.akka.metrics.MetricsWrapper 5 | import org.apache.spark.{SparkContext, SparkEnv} 6 | import org.joda.time.DateTime 7 | import spark.jobserver.io.JobDAO 8 | import spark.jobserver.util.{ContextURLClassLoader, LRUCache} 9 | 10 | case class JobJarInfo(constructor: () => SparkJob, 11 | className: String, 12 | jarFilePath: String) 13 | 14 | /** 15 | * A cache for SparkJob classes. A lot of times jobs are run repeatedly, and especially for low-latency 16 | * jobs, why retrieve the jar and load it every single time? 17 | */ 18 | class JobCache(maxEntries: Int, dao: JobDAO, sparkContext: SparkContext, loader: ContextURLClassLoader) { 19 | private val cache = new LRUCache[(String, DateTime, String), JobJarInfo](getClass, maxEntries) 20 | 21 | /** 22 | * Retrieves the given SparkJob class from the cache if it's there, otherwise use the DAO to retrieve it. 23 | * @param appName the appName under which the jar was uploaded 24 | * @param uploadTime the upload time for the version of the jar wanted 25 | * @param classPath the fully qualified name of the class/object to load 26 | */ 27 | def getSparkJob(appName: String, uploadTime: DateTime, classPath: String): JobJarInfo = { 28 | 29 | cache.get((appName, uploadTime, classPath), { 30 | val jarFilePath = new java.io.File(dao.retrieveJarFile(appName, uploadTime)).getAbsolutePath() 31 | sparkContext.addJar(jarFilePath) // Adds jar for remote executors 32 | loader.addURL(new URL("file:" + jarFilePath)) // Now jar added for local loader 33 | val constructor = JarUtils.loadClassOrObject[SparkJob](classPath, loader) 34 | JobJarInfo(constructor, classPath, jarFilePath) 35 | }) 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/JobInfoActor.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.ActorRef 4 | import akka.pattern.ask 5 | import akka.util.Timeout 6 | import com.typesafe.config.Config 7 | import ooyala.common.akka.InstrumentedActor 8 | import scala.concurrent.Await 9 | import spark.jobserver.ContextSupervisor.{GetContext, GetAdHocContext} 10 | import spark.jobserver.io.JobDAO 11 | 12 | object JobInfoActor { 13 | // Requests 14 | case class GetJobStatuses(limit: Option[Int]) 15 | case class GetJobStatus(jobId: String) 16 | case class GetJobConfig(jobId: String) 17 | case class StoreJobConfig(jobId: String, jobConfig: Config) 18 | 19 | // Responses 20 | case object JobConfigStored 21 | } 22 | 23 | class JobInfoActor(jobDao: JobDAO, contextSupervisor: ActorRef) extends InstrumentedActor { 24 | import CommonMessages._ 25 | import JobInfoActor._ 26 | import scala.concurrent.duration._ 27 | import scala.util.control.Breaks._ 28 | import context.dispatcher // for futures to work 29 | 30 | // Used in the asks (?) below to request info from contextSupervisor and resultActor 31 | implicit val ShortTimeout = Timeout(3 seconds) 32 | 33 | override def wrappedReceive: Receive = { 34 | case GetJobStatuses(limit) => 35 | if (limit.isDefined) { 36 | sender ! jobDao.getJobInfosLimit(limit.get).values.toSeq.sortBy(_.startTime.toString()) 37 | } else { 38 | sender ! jobDao.getJobInfos.values.toSeq.sortBy(_.startTime.toString()) 39 | } 40 | 41 | case GetJobStatus(jobId) => 42 | val jobInfoOpt = jobDao.getJobInfo(jobId) 43 | if (!jobInfoOpt.isDefined) { 44 | sender ! NoSuchJobId 45 | } else { 46 | sender ! jobInfoOpt.get 47 | } 48 | 49 | case GetJobResult(jobId) => 50 | breakable { 51 | val jobInfoOpt = jobDao.getJobInfo(jobId) 52 | if (!jobInfoOpt.isDefined) { 53 | sender ! NoSuchJobId 54 | break 55 | } 56 | 57 | jobInfoOpt.filter { job => job.isRunning || job.isErroredOut } 58 | .foreach { jobInfo => 59 | sender ! jobInfo 60 | break 61 | } 62 | 63 | // get the context from jobInfo 64 | val context = jobInfoOpt.get.contextName 65 | 66 | val future = (contextSupervisor ? ContextSupervisor.GetResultActor(context)).mapTo[ActorRef] 67 | val resultActor = Await.result(future, 3 seconds) 68 | 69 | val receiver = sender // must capture the sender since callbacks are run in a different thread 70 | for (result <- (resultActor ? GetJobResult(jobId))) { 71 | receiver ! result // a JobResult(jobId, result) object is sent 72 | } 73 | } 74 | 75 | case GetJobConfig(jobId) => 76 | sender ! jobDao.getJobConfigs.get(jobId).getOrElse(NoSuchJobId) 77 | 78 | case StoreJobConfig(jobId, jobConfig) => 79 | jobDao.saveJobConfig(jobId, jobConfig) 80 | sender ! JobConfigStored 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/JobResultActor.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.ActorRef 4 | import ooyala.common.akka.InstrumentedActor 5 | import ooyala.common.akka.metrics.MetricsWrapper 6 | import spark.jobserver.util.GuavaCacheUtils._ 7 | 8 | import scala.collection.mutable 9 | 10 | /** 11 | * It is an actor to manage results that are returned from jobs. 12 | * 13 | * TODO: support multiple subscribers for same JobID 14 | */ 15 | class JobResultActor extends InstrumentedActor { 16 | import CommonMessages._ 17 | 18 | private val config = context.system.settings.config 19 | 20 | private val cache = buildCache[String, Any](config).withMetrics(getClass) 21 | 22 | private val subscribers = mutable.HashMap.empty[String, ActorRef] // subscribers 23 | 24 | // metrics 25 | private val metricSubscribers = 26 | MetricsWrapper.newGauge(getClass, "subscribers-size", subscribers.size) 27 | 28 | def wrappedReceive: Receive = { 29 | case Subscribe(jobId, receiver, events) => 30 | if (events.contains(classOf[JobResult])) { 31 | subscribers(jobId) = receiver 32 | logger.info("Added receiver {} to subscriber list for JobID {}", receiver, jobId: Any) 33 | } 34 | 35 | case Unsubscribe(jobId, receiver) => 36 | if (!subscribers.contains(jobId)) { 37 | sender ! NoSuchJobId 38 | } else { 39 | subscribers.remove(jobId) 40 | logger.info("Removed subscriber list for JobID {}", jobId) 41 | } 42 | 43 | case GetJobResult(jobId) => 44 | sender ! Option(cache.getIfPresent(jobId)).map(JobResult(jobId, _)).getOrElse(NoSuchJobId) 45 | 46 | case JobResult(jobId, result) => 47 | cache.put(jobId, result) 48 | logger.debug("Received job results for JobID {}", jobId) 49 | subscribers.get(jobId).foreach(_ ! JobResult(jobId, result)) 50 | subscribers.remove(jobId) 51 | } 52 | } -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/JobServer.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.ActorSystem 4 | import akka.actor.Props 5 | import com.typesafe.config.{Config, ConfigFactory} 6 | import java.io.File 7 | import ooyala.common.akka.metrics.{DatadogConfigParser, MetricsLevel, MetricsWrapper} 8 | import org.slf4j.LoggerFactory 9 | import scala.util.Try 10 | import spark.jobserver.io.{JobDAO, JobDAOMetricsMonitor} 11 | 12 | /** 13 | * The Spark Job Server is a web service that allows users to submit and run Spark jobs, check status, 14 | * and view results. 15 | * It may offer other goodies in the future. 16 | * It only takes in one optional command line arg, a config file to override the default (and you can still 17 | * use -Dsetting=value to override) 18 | * -- Configuration -- 19 | * {{{ 20 | * spark { 21 | * master = "local" 22 | * jobserver { 23 | * port = 8090 24 | * } 25 | * } 26 | * }}} 27 | */ 28 | object JobServer { 29 | val logger = LoggerFactory.getLogger(getClass) 30 | 31 | // Allow custom function to create ActorSystem. An example of why this is useful: 32 | // we can have something that stores the ActorSystem so it could be shut down easily later. 33 | def start(args: Array[String], makeSystem: Config => ActorSystem) { 34 | val defaultConfig = ConfigFactory.load() 35 | val config = if (args.length > 0) { 36 | val configFile = new File(args(0)) 37 | if (!configFile.exists()) { 38 | println("Could not find configuration file " + configFile) 39 | sys.exit(1) 40 | } 41 | ConfigFactory.parseFile(configFile).withFallback(defaultConfig) 42 | } else { 43 | defaultConfig 44 | } 45 | logger.info("Starting JobServer with config {}", config.getConfig("spark").root.render()) 46 | val port = config.getInt("spark.jobserver.port") 47 | 48 | // TODO: Hardcode for now to get going. Make it configurable later. 49 | val system = makeSystem(config) 50 | val clazz = Class.forName(config.getString("spark.jobserver.jobdao")) 51 | val ctor = clazz.getDeclaredConstructor(Class.forName("com.typesafe.config.Config")) 52 | val backingJobDao = ctor.newInstance(config).asInstanceOf[JobDAO] 53 | // Monitors JobDAO metrics if metrics level is appropriate. 54 | val metricsLevel = Try(MetricsLevel.valueOf(config.getInt("spark.jobserver.metrics.level"))) 55 | .getOrElse(MetricsLevel.NONE) 56 | val jobDAO = metricsLevel match { 57 | case MetricsLevel.NONE => backingJobDao 58 | case level => JobDAOMetricsMonitor.newInstance(backingJobDao, level) 59 | } 60 | 61 | val jarManager = system.actorOf(Props(classOf[JarManager], jobDAO), "jar-manager") 62 | val supervisor = system.actorOf(Props(classOf[LocalContextSupervisorActor], jobDAO), "context-supervisor") 63 | val jobInfo = system.actorOf(Props(classOf[JobInfoActor], jobDAO, supervisor), "job-info") 64 | val sparkWebUi = system.actorOf(Props(classOf[SparkWebUiActor]), "spark-web-ui") 65 | // Create initial contexts 66 | supervisor ! ContextSupervisor.AddContextsFromConfig 67 | new WebApi(system, config, port, jarManager, supervisor, jobInfo, sparkWebUi).start() 68 | 69 | // Setups and starts datadog reporting 70 | val datadogConfig = DatadogConfigParser.parse(config) 71 | MetricsWrapper.startDatadogReporter(datadogConfig) 72 | } 73 | 74 | def main(args: Array[String]) { 75 | start(args, config => ActorSystem("JobServer", config)) 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/JobServerNamedRdds.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.ActorRef 4 | import akka.util.Timeout 5 | import org.apache.spark.rdd.RDD 6 | import org.apache.spark.storage.StorageLevel 7 | import scala.concurrent.Await 8 | 9 | /** 10 | * An implementation of [[NamedRddSupport]] API for the Job Server. 11 | * Note that this contains code that executes on the same thread as the job. 12 | * Another part of this system is the rddManager, which is an actor which manages the concurrent 13 | * update of shared RDD state. 14 | */ 15 | class JobServerNamedRdds(val rddManager: ActorRef) extends NamedRdds { 16 | import RddManagerActorMessages._ 17 | 18 | require(rddManager != null, "rddManager ActorRef must not be null!") 19 | 20 | def getOrElseCreate[T](name: String, rddGen: => RDD[T]) 21 | (implicit timeout: Timeout = defaultTimeout): RDD[T] = { 22 | import akka.pattern.ask 23 | 24 | val future = rddManager.ask(GetOrElseCreateRddRequest(name))(timeout) 25 | val result: RDD[T] = Await.result(future, timeout.duration) match { 26 | case Left(error: Throwable) => 27 | throw new RuntimeException("Failed to get named RDD '" + name + "'", error) 28 | case Right(rdd: RDD[T]) => refreshRdd(rdd) 29 | case None => 30 | // Try to generate the RDD and send the result of the operation to the rddManager. 31 | try { 32 | val rdd = createRdd(rddGen, name) 33 | rddManager ! CreateRddResult(name, Right(rdd)) 34 | rdd 35 | } catch { 36 | case error: Throwable => 37 | rddManager ! CreateRddResult(name, Left(error)) 38 | throw new RuntimeException("Failed to create named RDD '" + name + "'", error) 39 | } 40 | } 41 | result 42 | } 43 | 44 | def get[T](name: String)(implicit timeout: Timeout = defaultTimeout): Option[RDD[T]] = { 45 | import akka.pattern.ask 46 | 47 | val future = rddManager ? GetRddRequest(name) 48 | Await.result(future, timeout.duration) match { 49 | case rddOpt: Option[RDD[T]] @unchecked => rddOpt.map { rdd => refreshRdd(rdd) } 50 | } 51 | } 52 | 53 | def update[T](name: String, rddGen: => RDD[T]): RDD[T] = { 54 | val rdd = createRdd(rddGen, name) 55 | rddManager ! CreateRddResult(name, Right(rdd)) 56 | rdd 57 | } 58 | 59 | def destroy(name: String) { 60 | rddManager ! DestroyRdd(name) 61 | } 62 | 63 | def getNames()(implicit timeout: Timeout = defaultTimeout): Iterable[String] = { 64 | import akka.pattern.ask 65 | 66 | val future = rddManager ? GetRddNames 67 | Await.result(future, timeout.duration) match { 68 | case answer: Iterable[String] @unchecked => answer 69 | } 70 | } 71 | 72 | /** 73 | * Creates an RDD by calling the given generator, sets its name, persists it with the given storage level, 74 | * and optionally forces its contents to be computed. 75 | * @param rddGen a 0-ary function which will be called to generate the RDD in the caller's thread. 76 | * @param name the name to assign to the RDD. 77 | * @param storageLevel the storage level to persist the RDD with. Default: StorageLevel.MEMORY_ONLY. 78 | * @param forceComputation if true, forces the RDD to be computed by calling count(). 79 | * @throws java.lang.IllegalArgumentException if forceComputation == true && 80 | * storageLevel == StorageLevel.NONE 81 | */ 82 | private def createRdd[T](rddGen: => RDD[T], 83 | name: String, 84 | forceComputation: Boolean = true, 85 | storageLevel: StorageLevel = defaultStorageLevel): RDD[T] = { 86 | require(!forceComputation || storageLevel != StorageLevel.NONE, 87 | "forceComputation implies storageLevel != NONE") 88 | val rdd = rddGen 89 | rdd.setName(name) 90 | if (storageLevel != StorageLevel.NONE) rdd.persist(storageLevel) 91 | // TODO: figure out if there is a better way to force the RDD to be computed 92 | if (forceComputation) rdd.count() 93 | rdd 94 | } 95 | 96 | /** Calls rdd.persist(), which updates the RDD's cached timestamp, meaning it won't get 97 | * garbage collected by Spark for some time. 98 | * @param rdd the RDD 99 | */ 100 | private def refreshRdd[T](rdd: RDD[T]): RDD[T] = rdd.persist(rdd.getStorageLevel) 101 | } 102 | -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/JobStatusActor.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.ActorRef 4 | import com.codahale.metrics.Meter 5 | import ooyala.common.akka.metrics.MetricsWrapper 6 | import ooyala.common.akka.InstrumentedActor 7 | import scala.collection.mutable 8 | import scala.util.Try 9 | import spark.jobserver.io.{ JobInfo, JobDAO } 10 | 11 | object JobStatusActor { 12 | case class JobInit(jobInfo: JobInfo) 13 | case class GetRunningJobStatus() 14 | } 15 | 16 | /** 17 | * It is an actor to manage job status updates 18 | * 19 | */ 20 | class JobStatusActor(jobDao: JobDAO) extends InstrumentedActor { 21 | import CommonMessages._ 22 | import JobStatusActor._ 23 | import spark.jobserver.util.DateUtils.dateTimeToScalaWrapper 24 | 25 | // jobId to its JobInfo 26 | private val infos = new mutable.HashMap[String, JobInfo] 27 | // subscribers 28 | private val subscribers = new mutable.HashMap[String, mutable.MultiMap[Class[_], ActorRef]] 29 | 30 | // metrics 31 | val metricNumSubscriptions = MetricsWrapper.newGauge(getClass, "num-subscriptions", subscribers.size) 32 | val metricNumJobInfos = MetricsWrapper.newGauge(getClass, "num-running-jobs", infos.size) 33 | val metricJobSubmissionRate = MetricsWrapper.newMeter(getClass, "job-submission-rate") 34 | val metricStatusRates = mutable.HashMap.empty[String, Meter] 35 | 36 | // timer for job latency 37 | private val jobLatencyTimer = MetricsWrapper.newTimer(getClass, "job-latency"); 38 | // timer context to measure the job latency (jobId to Timer.Context mapping) 39 | private val latencyTimerContextMap = new mutable.HashMap[String, com.codahale.metrics.Timer.Context] 40 | 41 | override def wrappedReceive: Receive = { 42 | case GetRunningJobStatus => 43 | sender ! infos.values.toSeq.sortBy(_.startTime) // TODO(kelvinchu): Use toVector instead in Scala 2.10 44 | 45 | case Unsubscribe(jobId, receiver) => 46 | subscribers.get(jobId) match { 47 | case Some(jobSubscribers) => 48 | jobSubscribers.transform { case (event, receivers) => receivers -= receiver } 49 | .retain { case (event, receivers) => receivers.nonEmpty } 50 | if (jobSubscribers.isEmpty) subscribers.remove(jobId) 51 | case None => 52 | // TODO: The message below is named poorly. There may be such a job id, there are just no 53 | // registered subscribers for this job id. 54 | logger.error("No such job id " + jobId) 55 | sender ! NoSuchJobId 56 | } 57 | 58 | case Subscribe(jobId, receiver, events) => 59 | // Subscription is independent of job life cycles. So, don't need to check infos. 60 | val jobSubscribers = subscribers.getOrElseUpdate(jobId, newMultiMap()) 61 | events.foreach { event => jobSubscribers.addBinding(event, receiver) } 62 | 63 | case JobInit(jobInfo) => 64 | // TODO (kelvinchu): Check if the jobId exists in the persistence store already 65 | if (!infos.contains(jobInfo.jobId)) { 66 | infos(jobInfo.jobId) = jobInfo 67 | metricJobSubmissionRate.mark() 68 | } else { 69 | sender ! JobInitAlready 70 | } 71 | 72 | case msg: JobStarted => 73 | latencyTimerContextMap(msg.jobId) = jobLatencyTimer.time(); 74 | processStatus(msg, "started") { 75 | case (info, msg) => 76 | info.copy(startTime = msg.startTime) 77 | } 78 | 79 | case msg: JobFinished => 80 | stopTimer(msg.jobId) 81 | processStatus(msg, "finished OK", remove = true) { 82 | case (info, msg) => 83 | info.copy(endTime = Some(msg.endTime)) 84 | } 85 | 86 | case msg: JobValidationFailed => 87 | processStatus(msg, "validation failed", remove = true) { 88 | case (info, msg) => 89 | info.copy(endTime = Some(msg.endTime), error = Some(msg.err)) 90 | } 91 | 92 | case msg: JobErroredOut => 93 | stopTimer(msg.jobId) 94 | processStatus(msg, "finished with an error", remove = true) { 95 | case (info, msg) => 96 | info.copy(endTime = Some(msg.endTime), error = Some(msg.err)) 97 | } 98 | } 99 | 100 | private def stopTimer(jobId: String) { 101 | latencyTimerContextMap.get(jobId).foreach { timerContext => 102 | timerContext.stop() 103 | latencyTimerContextMap.remove(jobId) 104 | } 105 | } 106 | 107 | private def processStatus[M <: StatusMessage](msg: M, logMessage: String, remove: Boolean = false) 108 | (infoModifier: (JobInfo, M) => JobInfo) { 109 | if (infos.contains(msg.jobId)) { 110 | infos(msg.jobId) = infoModifier(infos(msg.jobId), msg) 111 | logger.info("Job {} {}", msg.jobId: Any, logMessage) 112 | jobDao.saveJobInfo(infos(msg.jobId)) 113 | publishMessage(msg.jobId, msg) 114 | updateMessageRate(msg) 115 | if (remove) infos.remove(msg.jobId) 116 | } else { 117 | logger.error("No such job id " + msg.jobId) 118 | sender ! NoSuchJobId 119 | } 120 | } 121 | 122 | private def updateMessageRate(msg: StatusMessage) { 123 | val msgClass = msg.getClass.getCanonicalName 124 | 125 | lazy val getShortName = Try(msgClass.split('.').last).toOption.getOrElse(msgClass) 126 | 127 | metricStatusRates.getOrElseUpdate(msgClass, MetricsWrapper.newMeter(getClass, getShortName + ".messages")).mark() 128 | } 129 | 130 | private def publishMessage(jobId: String, message: StatusMessage) { 131 | for ( 132 | jobSubscribers <- subscribers.get(jobId); 133 | receivers <- jobSubscribers.get(message.getClass); 134 | receiver <- receivers 135 | ) { 136 | receiver ! message 137 | } 138 | } 139 | 140 | private def newMultiMap(): mutable.MultiMap[Class[_], ActorRef] = 141 | new mutable.HashMap[Class[_], mutable.Set[ActorRef]] with mutable.MultiMap[Class[_], ActorRef] 142 | } 143 | -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/RddManagerActor.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.ActorRef 4 | import ooyala.common.akka.InstrumentedActor 5 | import ooyala.common.akka.metrics.MetricsWrapper 6 | import org.apache.spark.rdd.RDD 7 | import org.apache.spark.SparkContext 8 | import scala.collection.mutable 9 | 10 | object RddManagerActorMessages { 11 | // Message which asks to retrieve an RDD by name. If no such RDD is found, None will be returned. 12 | case class GetRddRequest(name: String) 13 | 14 | // Message which asks to retrieve an RDD by name. Different from GetRddRequest, because it tells the 15 | // RddManager that the client is willing to create the RDD with this name if one does not already exist. 16 | case class GetOrElseCreateRddRequest(name: String) 17 | 18 | // Message which tells the RddManager that a new RDD has been created, or that RDD generation failed. 19 | case class CreateRddResult(name: String, rddOrError: Either[Throwable, RDD[_]]) 20 | 21 | // Message which tells the RddManager that an RDD should be destroyed and all of its cached blocks removed 22 | case class DestroyRdd(name: String) 23 | 24 | // Message which asks for the names of all RDDs currently managed by the RddManager 25 | case object GetRddNames 26 | } 27 | 28 | class RddManagerActor(sparkContext: SparkContext) extends InstrumentedActor { 29 | import RddManagerActorMessages._ 30 | 31 | private val namesToIds = new mutable.HashMap[String, Int]() 32 | private val waiters = 33 | new mutable.HashMap[String, mutable.Set[ActorRef]] with mutable.MultiMap[String, ActorRef] 34 | private val inProgress = mutable.Set[String]() 35 | // Metrics 36 | private val rddNumGauge = MetricsWrapper.newGauge(getClass, "num-rdds", namesToIds.size) 37 | 38 | def wrappedReceive: Receive = { 39 | case GetRddRequest(name) => sender ! getExistingRdd(name) 40 | 41 | case GetOrElseCreateRddRequest(name) if inProgress.contains(name) => 42 | logger.info("RDD [{}] already being created, actor {} added to waiters list", name: Any, sender.path) 43 | waiters.addBinding(name, sender) 44 | 45 | case GetOrElseCreateRddRequest(name) => getExistingRdd(name) match { 46 | case Some(rdd) => sender ! Right(rdd) 47 | case None => 48 | logger.info("RDD [{}] not found, starting creation", name) 49 | inProgress.add(name) 50 | sender ! None 51 | } 52 | 53 | // TODO: log the error? 54 | case CreateRddResult(name, Left(error)) => notifyAndClearWaiters(name, Left(error)) 55 | 56 | case CreateRddResult(name, Right(rdd)) => 57 | val oldRddOption = getExistingRdd(name) 58 | namesToIds(name) = rdd.id 59 | notifyAndClearWaiters(name, Right(rdd)) 60 | // Note: unpersist the old rdd we just replaced, if there was one 61 | if (oldRddOption.isDefined && oldRddOption.get.id != rdd.id) { 62 | oldRddOption.get.unpersist(blocking = false) 63 | } 64 | 65 | case DestroyRdd(name) => getExistingRdd(name).foreach { rdd => 66 | namesToIds.remove(name) 67 | rdd.unpersist(blocking = false) 68 | } 69 | 70 | case GetRddNames => 71 | val persistentRdds = sparkContext.getPersistentRDDs 72 | val result = namesToIds.collect { case (name, id) if persistentRdds.contains(id) => name } 73 | // optimization: can remove stale names from our map if the SparkContext has unpersisted them. 74 | (namesToIds.keySet -- result).foreach { staleName => namesToIds.remove(staleName) } 75 | sender ! result 76 | } 77 | 78 | private def getExistingRdd(name: String): Option[RDD[_]] = 79 | namesToIds.get(name).flatMap { id => sparkContext.getPersistentRDDs.get(id) } match { 80 | case Some(rdd) => Some(rdd) 81 | case None => 82 | // If this happens, maybe we never knew about this RDD, or maybe we had a name -> id mapping, but 83 | // spark's MetadataCleaner has evicted this RDD from the cache because it was too old, and we need 84 | // to forget about it. Remove it from our names -> ids map and respond as if we never knew about it. 85 | namesToIds.remove(name) 86 | None 87 | } 88 | 89 | private def notifyAndClearWaiters(name: String, message: Any) { 90 | waiters.get(name).foreach { actors => actors.foreach { actor => actor ! message } } 91 | waiters.remove(name) // Note: this removes all bindings for the key in the MultiMap 92 | inProgress.remove(name) // this RDD is no longer being computed, clear in progress flag 93 | } 94 | } 95 | -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/SparkWebUiActor.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.ActorSystem 4 | import akka.io.IO 5 | import akka.pattern.ask 6 | import akka.util.Timeout 7 | import ooyala.common.akka.InstrumentedActor 8 | import scala.util.{Success, Failure} 9 | import scala.concurrent.Future 10 | import spark.jobserver.SparkWebUiActor.{SparkWorkersErrorInfo, SparkWorkersInfo, GetWorkerStatus} 11 | import spray.can.Http 12 | import spray.client.pipelining.{Get, sendReceive, SendReceive} 13 | import spray.http.{HttpResponse, HttpRequest} 14 | 15 | object SparkWebUiActor { 16 | // Requests 17 | case class GetWorkerStatus() 18 | 19 | // Responses 20 | case class SparkWorkersInfo(alive: Int, dead: Int) 21 | case class SparkWorkersErrorInfo(message :String) 22 | } 23 | /** 24 | * This actor pulls Spark worker status info (ALIVE, DEAD etc) from Spark admin web ui 25 | * Collecting worker info from HTML page is not ideal. 26 | * But at this time Spark does not provide public API yet to expose worker status. 27 | * Also, the current implementation only works for Spark standalone mode 28 | */ 29 | class SparkWebUiActor extends InstrumentedActor { 30 | import actorSystem.dispatcher // execution context for futures 31 | import scala.concurrent.duration._ 32 | 33 | implicit val actorSystem: ActorSystem = context.system 34 | 35 | val config = context.system.settings.config 36 | 37 | val sparkWebHostUrls: Array[String] = getSparkHostName() 38 | val sparkWebHostPort = config.getInt("spark.webUrlPort") 39 | 40 | // implicit timeout value for ? of IO(Http) 41 | implicit val shortTimeout = Timeout(3 seconds) 42 | // get a pipeline every time we need it since pipeline could time out or die. 43 | // The connector will be re-used if it exists so the cost is low: 44 | // from http://spray.io/documentation/1.1-M8/spray-can/http-client/host-level/ 45 | // If there is no connector actor running for the given combination of hostname, 46 | // port and settings spray-can will start a new one, 47 | // otherwise the existing one is going to be re-used. 48 | def pipelines: Array[Future[SendReceive]] = sparkWebHostUrls.map(url => 49 | for ( 50 | Http.HostConnectorInfo(connector, _) <- IO(Http) ? Http.HostConnectorSetup(url, port = sparkWebHostPort) 51 | ) yield sendReceive(connector) 52 | ) 53 | 54 | override def postStop() { 55 | logger.info("Shutting down actor system for SparkWebUiActor") 56 | } 57 | 58 | override def wrappedReceive: Receive = { 59 | case GetWorkerStatus() => 60 | val request = Get("/") 61 | 62 | val theSender = sender 63 | 64 | pipelines.map {pipeline => 65 | val responseFuture: Future[HttpResponse] = pipeline.flatMap(_(request)) 66 | responseFuture onComplete { 67 | case Success(httpResponse) => 68 | val content = httpResponse.entity.asString.replace('\n', ' '); 69 | 70 | // regex to detect the #running tasks 71 | val runningTaskRegex = """.*
  • Applications:\s*(\d+)\s+Running.*""".r 72 | content match { 73 | case runningTaskRegex(runningTaskStr) => 74 | if (runningTaskStr != null && runningTaskStr.length > 0 && runningTaskStr.toInt > 0) { 75 | // we believe it is a active master if it has active running tasks 76 | // we only check the workers on active master 77 | val aliveWorkerNum = "ALIVE".r.findAllIn(content).length 78 | val deadWorkerNum = "DEAD".r.findAllIn(content).length 79 | 80 | theSender ! SparkWorkersInfo(aliveWorkerNum, deadWorkerNum) 81 | } 82 | case _ => throw new RuntimeException("Could not parse HTML response: '" + content + "'") 83 | } 84 | case Failure(error) => 85 | val msg = "Failed to retrieve Spark web UI: " + error.getMessage 86 | logger.error(msg) 87 | } 88 | } 89 | } 90 | 91 | def getSparkHostName(): Array[String] = { 92 | val master = config.getString("spark.master") 93 | // Regular expression used for local[N] and local[*] master formats 94 | val LOCAL_N_REGEX = """local\[([0-9\*]+)\]""".r 95 | // Regular expression for connecting to Spark deploy clusters 96 | val SPARK_REGEX = """spark://(.*)""".r 97 | 98 | master match { 99 | case "localhost" | "local" | LOCAL_N_REGEX(_) => Array("localhost") 100 | case SPARK_REGEX(sparkUrl) => 101 | val masterUrls = sparkUrl.split(",").map(s => { 102 | val splits = s.split(":") 103 | splits(0) 104 | }) 105 | masterUrls 106 | case _ => throw new RuntimeException("Could not parse Master URL: '" + master + "'") 107 | } 108 | } 109 | 110 | 111 | } 112 | -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/io/JobCacheDAO.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver.io 2 | 3 | import java.io._ 4 | 5 | import com.typesafe.config._ 6 | import org.joda.time.DateTime 7 | import org.slf4j.LoggerFactory 8 | import spark.jobserver.util.LRUCache 9 | 10 | import scala.collection.mutable 11 | import scala.util.Try 12 | 13 | /** 14 | * Store job information (JobInfo, config) in memory cache only, but jar files still in file system. 15 | * 16 | * 17 | * @param config 18 | */ 19 | class JobCacheDAO(config: Config) extends JobDAO { 20 | private val logger = LoggerFactory.getLogger(getClass) 21 | 22 | private val charsetName = "UTF-8" 23 | 24 | // appName to its set of upload times. Decreasing times in the seq. 25 | private val apps = mutable.HashMap.empty[String, Seq[DateTime]] 26 | // jobId to its (JobInfo, config) 27 | private val jobs = new LRUCache[String, (JobInfo, Config)](this.getClass, 28 | Try(config.getInt("spark.jobserver.cachedao.size")).getOrElse(500)) 29 | 30 | private val rootDir = getOrElse(config.getString("spark.jobserver.cachedao.rootdir"), 31 | "/tmp/spark-jobserver/cachedao/data") 32 | private val rootDirFile = new File(rootDir) 33 | logger.info("rootDir is " + rootDirFile.getAbsolutePath) 34 | 35 | private val jarsFile = new File(rootDirFile, "jars.data") 36 | private var jarsOutputStream: DataOutputStream = null 37 | 38 | init() 39 | 40 | private def init() { 41 | // create the date directory if it doesn't exist 42 | if (!rootDirFile.exists()) { 43 | if (!rootDirFile.mkdirs()) { 44 | throw new RuntimeException("Could not create directory " + rootDir) 45 | } 46 | } 47 | 48 | // read back all apps info during startup 49 | if (jarsFile.exists()) { 50 | val in = new DataInputStream(new BufferedInputStream(new FileInputStream(jarsFile))) 51 | try { 52 | while (true) { 53 | val jarInfo = readJarInfo(in) 54 | addJar(jarInfo.appName, jarInfo.uploadTime) 55 | } 56 | } catch { 57 | case e: EOFException => // do nothing 58 | 59 | } finally { 60 | in.close() 61 | } 62 | } 63 | 64 | // Don't buffer the stream. I want the apps meta data log directly into the file. 65 | // Otherwise, server crash will lose the buffer data. 66 | jarsOutputStream = new DataOutputStream(new FileOutputStream(jarsFile, true)) 67 | } 68 | 69 | override def saveJar(appName: String, uploadTime: DateTime, jarBytes: Array[Byte]) { 70 | // The order is important. Save the jar file first and then log it into jobsFile. 71 | val outFile = new File(rootDir, createJarName(appName, uploadTime) + ".jar") 72 | val bos = new BufferedOutputStream(new FileOutputStream(outFile)) 73 | try { 74 | logger.debug("Writing {} bytes to file {}", jarBytes.size, outFile.getPath) 75 | bos.write(jarBytes) 76 | bos.flush() 77 | } finally { 78 | bos.close() 79 | } 80 | 81 | // log it into jobsFile 82 | writeJarInfo(jarsOutputStream, JarInfo(appName, uploadTime)) 83 | 84 | // track the new jar in memory 85 | addJar(appName, uploadTime) 86 | } 87 | 88 | private def writeJarInfo(out: DataOutputStream, jarInfo: JarInfo) { 89 | out.writeUTF(jarInfo.appName) 90 | out.writeLong(jarInfo.uploadTime.getMillis) 91 | } 92 | 93 | private def readJarInfo(in: DataInputStream) = JarInfo(in.readUTF, new DateTime(in.readLong)) 94 | 95 | private def addJar(appName: String, uploadTime: DateTime) { 96 | if (apps.contains(appName)) { 97 | apps(appName) = uploadTime +: apps(appName) // latest time comes first 98 | } else { 99 | apps(appName) = Seq(uploadTime) 100 | } 101 | } 102 | 103 | def getApps: Map[String, DateTime] = apps.map { 104 | case (appName, uploadTimes) => 105 | appName -> uploadTimes.head 106 | }.toMap 107 | 108 | override def retrieveJarFile(appName: String, uploadTime: DateTime): String = 109 | new File(rootDir, createJarName(appName, uploadTime) + ".jar").getAbsolutePath 110 | 111 | private def createJarName(appName: String, uploadTime: DateTime): String = 112 | appName + "-" + uploadTime.toString().replace(':', '_') 113 | 114 | override def saveJobInfo(jobInfo: JobInfo) { 115 | jobs.synchronized { 116 | jobs.get(jobInfo.jobId) match { 117 | case Some((_, config)) => jobs.put(jobInfo.jobId, (jobInfo, config)) 118 | case None => jobs.put(jobInfo.jobId, (jobInfo, null)) 119 | } 120 | } 121 | } 122 | 123 | private def readError(in: DataInputStream) = { 124 | val error = in.readUTF() 125 | if (error == "") None else Some(new Throwable(error)) 126 | } 127 | 128 | override def getJobInfos: Map[String, JobInfo] = jobs.toMap.map {case (id, (jobInfo, _)) => (id, jobInfo)} 129 | 130 | override def getJobInfosLimit(limit: Int): Map[String, JobInfo] = getJobInfos.takeRight(limit) 131 | 132 | override def getJobInfo(jobId: String): Option[JobInfo] = jobs.get(jobId).map(_._1) 133 | 134 | override def saveJobConfig(jobId: String, config: Config) { 135 | jobs.synchronized { 136 | jobs.get(jobId) match { 137 | case Some((jobInfo, _)) => jobs.put(jobId, (jobInfo, config)) 138 | case None => jobs.put(jobId, (null, config)) 139 | } 140 | } 141 | } 142 | 143 | override def getJobConfigs: Map[String, Config] = jobs.toMap.map {case (id, (_, config)) => (id, config)} 144 | 145 | } 146 | -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/io/JobDAO.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver.io 2 | 3 | import com.typesafe.config._ 4 | import org.joda.time.{ Duration, DateTime } 5 | 6 | // Uniquely identifies the jar used to run a job 7 | case class JarInfo(appName: String, uploadTime: DateTime) 8 | 9 | // Both a response and used to track job progress 10 | // NOTE: if endTime is not None, then the job has finished. 11 | case class JobInfo(jobId: String, contextName: String, 12 | jarInfo: JarInfo, classPath: String, 13 | startTime: DateTime, endTime: Option[DateTime], 14 | error: Option[Throwable]) { 15 | def jobLengthMillis: Option[Long] = endTime.map { end => new Duration(startTime, end).getMillis() } 16 | 17 | def isRunning: Boolean = !endTime.isDefined 18 | def isErroredOut: Boolean = endTime.isDefined && error.isDefined 19 | } 20 | 21 | /** 22 | * Core trait for data access objects for persisting data such as jars, applications, jobs, etc. 23 | */ 24 | trait JobDAO { 25 | /** 26 | * Persist a jar. 27 | * 28 | * @param appName 29 | * @param uploadTime 30 | * @param jarBytes 31 | */ 32 | def saveJar(appName: String, uploadTime: DateTime, jarBytes: Array[Byte]) 33 | 34 | /** 35 | * Return all applications name and their last upload times. 36 | * 37 | * @return 38 | */ 39 | def getApps: Map[String, DateTime] 40 | 41 | /** 42 | * TODO(kelvinchu): Remove this method later when JarManager doesn't use it anymore. 43 | * 44 | * @param appName 45 | * @param uploadTime 46 | * @return the local file path of the retrieved jar file. 47 | */ 48 | def retrieveJarFile(appName: String, uploadTime: DateTime): String 49 | 50 | /** 51 | * Persist a job info. 52 | * 53 | * @param jobInfo 54 | */ 55 | def saveJobInfo(jobInfo: JobInfo) 56 | 57 | /** 58 | * Return all job ids to their job info. 59 | * 60 | * @return 61 | */ 62 | def getJobInfos: Map[String, JobInfo] 63 | 64 | /** 65 | * Get jobInfo for the first limit job 66 | * 67 | * @param limit the number of jobInfo returned 68 | * @return Map[String, JobInfo] 69 | */ 70 | def getJobInfosLimit(limit: Int): Map[String, JobInfo] 71 | 72 | /** 73 | * Get jobInfo for a given job id 74 | * 75 | * @param jobId 76 | * @return Option[JobInfo] 77 | */ 78 | def getJobInfo(jobId: String): Option[JobInfo] 79 | 80 | /** 81 | * Persist a job configuration along with provided jobId. 82 | * 83 | * @param jobId 84 | * @param jobConfig 85 | */ 86 | def saveJobConfig(jobId: String, jobConfig: Config) 87 | 88 | /** 89 | * Return all job ids to their job configuration. 90 | * 91 | * @return 92 | */ 93 | def getJobConfigs: Map[String, Config] 94 | 95 | /** 96 | * Returns the last upload time for a given app name. 97 | * @return Some(lastUploadedTime) if the app exists and the list of times is nonempty, None otherwise 98 | */ 99 | def getLastUploadTime(appName: String): Option[DateTime] = 100 | getApps.get(appName) 101 | 102 | /** 103 | * Return all contexts 104 | */ 105 | def getContexts: Map[String, Config] = { 106 | Map() 107 | } 108 | 109 | /** 110 | * Return the context configuration 111 | * 112 | * @param name the context name 113 | */ 114 | def getContextConfig(name: String): Option[Config] = { 115 | None 116 | } 117 | 118 | /** 119 | * Persist a context configuration 120 | * 121 | * @param name the context name 122 | * @param config the context configuration 123 | */ 124 | def saveContextConfig(name: String, config: Config) = { 125 | } 126 | 127 | /** 128 | * A safe API for getting values from Typesafe Config, will return a default if the 129 | * value is missing. If the value is badly formatted, error still goes through. 130 | */ 131 | def getOrElse[T](getter: => T, default: T): T = { 132 | try getter catch { 133 | case e: ConfigException.Missing => default 134 | } 135 | } 136 | } -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/io/JobDAOMetricsMonitor.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver.io 2 | 3 | import com.codahale.metrics.Meter 4 | import com.typesafe.config.{Config, ConfigRenderOptions} 5 | import java.lang.reflect.{InvocationHandler, Method, Proxy} 6 | import ooyala.common.akka.metrics.MetricsLevel 7 | import ooyala.common.akka.metrics.MetricsWrapper.{newHistogram, newMeter} 8 | 9 | /** 10 | * This invocation handler monitors the job DAO and updates the metrics. 11 | * 12 | * @param jobDao the actual job DAO to be monitored 13 | * @param metricsLevel the level of details of metrics to monitor 14 | */ 15 | class JobDAOMetricsMonitor(jobDao: JobDAO, metricsLevel: MetricsLevel) extends InvocationHandler { 16 | // Basic metrics 17 | private val basicMetrics: Map[String, Meter] = { 18 | val metrics = collection.mutable.Map[String, Meter]() 19 | 20 | classOf[JobDAO].getMethods.foreach { 21 | method => { 22 | metrics(method.getName) = newMeter(getClass, method.getName) 23 | } 24 | } 25 | 26 | metrics.toMap 27 | } 28 | // Fine metrics 29 | private lazy val metricJarInfoSize = newHistogram(getClass, "JarSize") 30 | private lazy val metricJobInfoSize = newHistogram(getClass, "JobInfoSize") 31 | private lazy val metricJobConfigSize = newHistogram(getClass, "JobConfigSize") 32 | private lazy val metricContextConfigSize = newHistogram(getClass, "ContextConfigSize") 33 | // Handlers for updating fine metrics 34 | private val fineMetricsHandlers = Map[String, Array[AnyRef] => Unit]( 35 | "saveJar" -> ( 36 | (args: Array[AnyRef]) => { 37 | // Updates metric for jar size 38 | val appName = args(0).asInstanceOf[String] 39 | val jarBytes = args(2).asInstanceOf[Array[Byte]] 40 | metricJarInfoSize.update(appName.length + jarBytes.size) 41 | }), 42 | 43 | "saveJobConfig" -> ( 44 | (args: Array[AnyRef]) => { 45 | // Updates metric for job config size 46 | val jobConfig = args(1).asInstanceOf[Config] 47 | val configStr = jobConfig.root().render(ConfigRenderOptions.concise()) 48 | metricJobConfigSize.update(configStr.length) 49 | }), 50 | 51 | "saveJobInfo" -> ( 52 | (args: Array[AnyRef]) => { 53 | // Updates metric for job info size 54 | val jobInfo = args(0).asInstanceOf[JobInfo] 55 | val jobInfoSize = jobInfo.contextName.length + jobInfo.classPath.length + 56 | jobInfo.error.map(_.getMessage.length).getOrElse(0) 57 | metricJobInfoSize.update(jobInfoSize) 58 | }), 59 | 60 | "saveContextConfig" -> ( 61 | (args: Array[AnyRef]) => { 62 | // Updates metric for context config size 63 | val name = args(0).asInstanceOf[String] 64 | val config = args(1).asInstanceOf[Config] 65 | val configStr = config.root().render(ConfigRenderOptions.concise()) 66 | metricContextConfigSize.update(name.length + configStr.length) 67 | })) 68 | 69 | override def invoke(proxy: AnyRef, method: Method, args: Array[AnyRef]): AnyRef = { 70 | val methodName = method.getName 71 | 72 | // Monitors basic metrics 73 | basicMetrics.get(methodName).foreach { 74 | _.mark 75 | } 76 | 77 | // Monitors more detailed metrics 78 | if (metricsLevel >= MetricsLevel.FINE) { 79 | fineMetricsHandlers.get(methodName).foreach { 80 | _(args) 81 | } 82 | } 83 | 84 | method.invoke(jobDao, args: _*) 85 | } 86 | } 87 | 88 | object JobDAOMetricsMonitor { 89 | 90 | def newInstance(jobDao: JobDAO, metricsLevel: MetricsLevel): JobDAO = { 91 | val metricsMonitor = new JobDAOMetricsMonitor(jobDao, metricsLevel) 92 | 93 | return Proxy.newProxyInstance(getClass.getClassLoader, Array(classOf[JobDAO]), metricsMonitor) 94 | .asInstanceOf[JobDAO] 95 | } 96 | } -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/util/ContextURLClassLoader.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver.util 2 | 3 | import java.net.{URLClassLoader, URL} 4 | import org.slf4j.LoggerFactory 5 | 6 | /** 7 | * The addURL method in URLClassLoader is protected. We subclass it to make this accessible. 8 | * NOTE: This is copied from Spark's ExecutorURLClassLoader, which is private[spark]. 9 | */ 10 | class ContextURLClassLoader(urls: Array[URL], parent: ClassLoader) 11 | extends URLClassLoader(urls, parent) { 12 | 13 | val logger = LoggerFactory.getLogger(getClass) 14 | 15 | override def addURL(url: URL) { 16 | if (!getURLs.contains(url)) { 17 | super.addURL(url) 18 | logger.info("Added URL " + url + " to ContextURLClassLoader") 19 | } 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/util/DateUtils.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver.util 2 | 3 | import org.joda.time.format.ISODateTimeFormat 4 | import org.joda.time.{DateTime, DateTimeComparator, DateTimeZone} 5 | 6 | 7 | object DateUtils { 8 | val ZeroTime = dtFromUtcSeconds(0) 9 | 10 | private val iso8601format = ISODateTimeFormat.dateTimeNoMillis() 11 | private val iso8601withMillis = ISODateTimeFormat.dateTime() 12 | private val dateComparator = DateTimeComparator.getInstance() 13 | 14 | def iso8601(dt: DateTime, fractions: Boolean = false): String = 15 | if (fractions) iso8601withMillis.print(dt) else iso8601format.print(dt) 16 | 17 | @inline def dtFromUtcSeconds(seconds: Int): DateTime = new DateTime(seconds * 1000L, DateTimeZone.UTC) 18 | 19 | @inline def dtFromIso8601(isoString: String): DateTime = new DateTime(isoString, DateTimeZone.UTC) 20 | 21 | /** 22 | * Implicit conversions so we can use Scala comparison operators 23 | * with JodaTime's DateTime 24 | */ 25 | implicit def dateTimeToScalaWrapper(dt: DateTime): DateTimeWrapper = new DateTimeWrapper(dt) 26 | 27 | class DateTimeWrapper(dt: DateTime) extends Ordered[DateTime] with Ordering[DateTime] { 28 | def compare(that: DateTime): Int = dateComparator.compare(dt, that) 29 | def compare(a: DateTime, b: DateTime): Int = dateComparator.compare(a, b) 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/util/GuavaCacheUtils.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver.util 2 | 3 | import java.util.concurrent.TimeUnit 4 | 5 | import com.google.common.cache.{CacheBuilder, Cache => GuavaCache} 6 | import com.typesafe.config.Config 7 | import ooyala.common.akka.metrics.MetricsWrapper 8 | 9 | object GuavaCacheUtils { 10 | def buildCache[K,V](cacheSize: Int, ttlSeconds: Long): GuavaCache[K,V] = { 11 | CacheBuilder.newBuilder 12 | .maximumSize(cacheSize) 13 | .expireAfterAccess(ttlSeconds, TimeUnit.SECONDS) 14 | .recordStats() 15 | .build() 16 | .asInstanceOf[GuavaCache[K,V]] 17 | } 18 | 19 | def buildCache[K,V](config: Config): GuavaCache[K,V] = 20 | buildCache( 21 | config.getInt("spark.jobserver.job-result-cache-size"), 22 | config.getLong("spark.jobserver.job-result-cache-ttl-seconds") 23 | ) 24 | 25 | //pimp my cache 26 | class WrappedCache[K,V](cache: GuavaCache[K,V]) { 27 | def withMetrics(klass: Class[_]): GuavaCache[K,V] = { 28 | MetricsWrapper.newGauge(klass, "cache-hit", cache.stats.hitCount()) 29 | MetricsWrapper.newGauge(klass, "cache-miss", cache.stats.missCount()) 30 | MetricsWrapper.newGauge(klass, "cache-eviction", cache.stats.evictionCount()) 31 | MetricsWrapper.newGauge(klass, "cache-request", cache.stats.requestCount()) 32 | cache 33 | } 34 | } 35 | 36 | implicit def toWrappedCache[K,V](cache: GuavaCache[K,V]): WrappedCache[K,V] = new WrappedCache[K,V](cache) 37 | } -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/util/LRUCache.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver.util 2 | 3 | import java.util.Map.Entry 4 | 5 | import ooyala.common.akka.metrics.MetricsWrapper 6 | 7 | import scala.collection.JavaConversions._ 8 | 9 | /** 10 | * A convenience class to define a Least-Recently-Used Cache with a maximum size. 11 | * The oldest entries by time of last access will be removed when the number of entries exceeds 12 | * cacheSize. 13 | * For definitions of cacheSize and loadingFactor, see the docs for java.util.LinkedHashMap 14 | * @see LinkedHashMap 15 | */ 16 | class LRUCache[K, V](cacheHolderClass: Class[_], 17 | cacheSize: Int, 18 | loadingFactor: Float = 0.75F) { 19 | private val cache = { 20 | val initialCapacity = math.ceil(cacheSize / loadingFactor).toInt + 1 21 | new java.util.LinkedHashMap[K, V](initialCapacity, loadingFactor, true) { 22 | protected override def removeEldestEntry(p1: Entry[K, V]): Boolean = size() > cacheSize 23 | } 24 | } 25 | 26 | // metrics for the LRU cache hits and misses. 27 | private val metricHit = MetricsWrapper.newCounter(cacheHolderClass, "cache-hit") 28 | private val metricMiss = MetricsWrapper.newCounter(cacheHolderClass, "cache-miss") 29 | 30 | /** size of the cache. This is an exact number and runs in constant time */ 31 | def size: Int = cache.size() 32 | 33 | /** @return TRUE if the cache contains the key */ 34 | def containsKey(k: K): Boolean = cache.get(k) match { 35 | case null => 36 | metricMiss.inc 37 | false 38 | case _ => 39 | metricHit.inc 40 | true 41 | } 42 | 43 | /** @return the value in cache or load a new value into cache */ 44 | def get(k: K, v: => V): V = { 45 | cache.get(k) match { 46 | case null => 47 | val evaluatedV = v 48 | cache.put(k, evaluatedV) 49 | metricMiss.inc 50 | evaluatedV 51 | case vv => 52 | metricHit.inc 53 | vv 54 | } 55 | } 56 | 57 | def put(k: K, v: V): V = cache.put(k, v) 58 | 59 | def get(k: K): Option[V] = cache.get(k) match { 60 | case null => 61 | metricMiss.inc 62 | None 63 | case vv => 64 | metricHit.inc 65 | Some(vv) 66 | } 67 | 68 | def toMap = mapAsScalaMap(cache).toMap 69 | } -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/util/SparkContextFactory.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver.util 2 | 3 | import com.typesafe.config.Config 4 | import org.apache.spark.SparkContext 5 | 6 | /** 7 | * I know how to make SparkContexts! 8 | * My implementing classes can be dynamically loaded using classloaders to ensure that the entire 9 | * SparkContext has access to certain dynamically loaded classes, for example, job jars. 10 | */ 11 | trait SparkContextFactory { 12 | /** 13 | * Creates a SparkContext. 14 | * @param config the overall system / job server Typesafe Config 15 | * @param contextConfig the config specific to this particular job 16 | * @param contextName the name of the context to start 17 | */ 18 | def makeContext(config: Config, contextConfig: Config, contextName: String): SparkContext 19 | } 20 | 21 | /** 22 | * The default factory creates a standard SparkContext. 23 | * In the future if we want to add additional methods, etc. then we can have additional factories. 24 | * For example a specialized SparkContext to manage RDDs in a user-defined way. 25 | * 26 | * If you create your own SparkContextFactory, please make sure it has zero constructor args. 27 | */ 28 | class DefaultSparkContextFactory extends SparkContextFactory { 29 | import SparkJobUtils._ 30 | 31 | def makeContext(config: Config, contextConfig: Config, contextName: String): SparkContext = { 32 | val conf = configToSparkConf(config, contextConfig, contextName) 33 | 34 | new SparkContext(conf) 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /job-server/src/spark.jobserver/util/SparkJobUtils.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver.util 2 | 3 | import com.typesafe.config.Config 4 | import org.apache.spark.SparkConf 5 | import scala.util.Try 6 | 7 | /** 8 | * Holds a few functions common to Job Server SparkJob's and SparkContext's 9 | */ 10 | object SparkJobUtils { 11 | import collection.JavaConverters._ 12 | 13 | /** 14 | * Creates a SparkConf for initializing a SparkContext based on various configs. 15 | * Note that anything in contextConfig with keys beginning with spark. get 16 | * put directly in the SparkConf. 17 | * 18 | * @param config the overall Job Server configuration (Typesafe Config) 19 | * @param contextConfig the Typesafe Config specific to initializing this context 20 | * (typically based on particular context/job) 21 | * @param contextName the context name 22 | * @return a SparkConf with everything properly configured 23 | */ 24 | def configToSparkConf(config: Config, contextConfig: Config, 25 | contextName: String): SparkConf = { 26 | val conf = new SparkConf() 27 | conf.setMaster(config.getString("spark.master")) 28 | .setAppName(contextName) 29 | 30 | for (cores <- Try(contextConfig.getInt("num-cpu-cores"))) { 31 | conf.set("spark.cores.max", cores.toString) 32 | } 33 | // Should be a -Xmx style string eg "512m", "1G" 34 | for (nodeMemStr <- Try(contextConfig.getString("memory-per-node"))) { 35 | conf.set("spark.executor.memory", nodeMemStr) 36 | } 37 | 38 | Try(config.getString("spark.home")).foreach { home => conf.setSparkHome(home) } 39 | 40 | // Set the Jetty port to 0 to find a random port 41 | conf.set("spark.ui.port", "0") 42 | 43 | // Set number of akka threads 44 | // TODO: need to figure out how many extra threads spark needs, besides the job threads 45 | conf.set("spark.akka.threads", (getMaxRunningJobs(config) + 4).toString) 46 | 47 | // Set any other settings in context config that start with "spark" 48 | for (e <- contextConfig.entrySet().asScala if e.getKey.startsWith("spark.")) { 49 | conf.set(e.getKey, e.getValue.unwrapped.toString) 50 | } 51 | 52 | conf 53 | } 54 | 55 | /** 56 | * Returns the maximum number of jobs that can run at the same time 57 | */ 58 | def getMaxRunningJobs(config: Config): Int = { 59 | val cpuCores = Runtime.getRuntime.availableProcessors 60 | Try(config.getInt("spark.jobserver.max-jobs-per-context")).getOrElse(cpuCores) 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /job-server/src/test/resources/local.test.jobcachedao.conf: -------------------------------------------------------------------------------- 1 | # This file contains test settings for: JobSqlDAOSpec 2 | 3 | spark.jobserver { 4 | # Number of job results to keep per JobResultActor/context 5 | job-result-cache-size = 5000 6 | 7 | job-result-cache-ttl-seconds = 600 8 | 9 | jobdao = spark.jobserver.io.JobCacheDAO 10 | 11 | cachedao { 12 | rootdir = /tmp/spark-jobserver-test/cachedao/data 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /job-server/src/test/resources/local.test.jobfiledao.conf: -------------------------------------------------------------------------------- 1 | # This file contains test settings for: JobSqlDAOSpec 2 | 3 | spark.jobserver { 4 | # Number of job results to keep per JobResultActor/context 5 | job-result-cache-size = 5000 6 | 7 | job-result-cache-ttl-seconds = 600 8 | 9 | jobdao = spark.jobserver.io.JobFileDAO 10 | 11 | filedao { 12 | rootdir = /tmp/spark-jobserver-test/filedao/data 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /job-server/src/test/resources/local.test.jobsqldao.conf: -------------------------------------------------------------------------------- 1 | # This file contains test settings for: JobSqlDAOSpec 2 | 3 | spark.jobserver { 4 | jobdao = spark.jobserver.io.JobSqlDAO 5 | 6 | sqldao { 7 | rootdir = /tmp/spark-job-server-test/sqldao/data 8 | # https://coderwall.com/p/a2vnxg 9 | h2 { 10 | url = "jdbc:h2:mem:jobserver-test;DATABASE_TO_UPPER=false;DB_CLOSE_DELAY=-1" 11 | } 12 | } 13 | } 14 | -------------------------------------------------------------------------------- /job-server/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Default LOG4J properties logs to a file and overwrites it every time. Works well for tests. 2 | 3 | # Root logger option 4 | log4j.rootLogger=INFO, LOGFILE 5 | 6 | log4j.appender.LOGFILE=org.apache.log4j.FileAppender 7 | log4j.appender.LOGFILE.File=job-server-test.log 8 | log4j.appender.LOGFILE.layout=org.apache.log4j.PatternLayout 9 | # log4j.appender.LOGFILE.layout.ConversionPattern=%d %-5p %c - %m%n 10 | log4j.appender.LOGFILE.layout.ConversionPattern=[%d] %-5p %.26c [%X{testName}] [%X{akkaSource}] - %m%n 11 | log4j.appender.LOGFILE.Append=false 12 | -------------------------------------------------------------------------------- /job-server/src/test/resources/logback-test.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | job-server-test.log 7 | 8 | 9 | false 10 | 11 | 12 | [%date{ISO8601}] %-5level %logger{26} [%X{akkaSource}] - %msg%n 13 | 14 | 15 | 16 | 17 | 18 | 19 | 20 | 21 | -------------------------------------------------------------------------------- /job-server/test/spark.jobserver/InMemoryDAO.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import com.typesafe.config.Config 4 | import java.io.{BufferedOutputStream, FileOutputStream} 5 | import org.joda.time.DateTime 6 | import scala.collection.mutable 7 | import spark.jobserver.io.{JobDAO, JobInfo} 8 | 9 | /** 10 | * In-memory DAO for easy unit testing 11 | */ 12 | class InMemoryDAO extends JobDAO { 13 | val jars = mutable.HashMap.empty[(String, DateTime), Array[Byte]] 14 | 15 | def saveJar(appName: String, uploadTime: DateTime, jarBytes: Array[Byte]) { 16 | jars((appName, uploadTime)) = jarBytes 17 | } 18 | 19 | def getApps(): Map[String, DateTime] = { 20 | jars.keys 21 | .groupBy(_._1) 22 | .map { case (appName, appUploadTimeTuples) => 23 | appName -> appUploadTimeTuples.map(_._2).toSeq.head 24 | }.toMap 25 | } 26 | 27 | def retrieveJarFile(appName: String, uploadTime: DateTime): String = { 28 | // Write the jar bytes to a temporary file 29 | val outFile = java.io.File.createTempFile("InMemoryDAO", ".jar") 30 | outFile.deleteOnExit() 31 | val bos = new BufferedOutputStream(new FileOutputStream(outFile)) 32 | try { 33 | bos.write(jars((appName, uploadTime))) 34 | } finally { 35 | bos.close() 36 | } 37 | outFile.getAbsolutePath() 38 | } 39 | 40 | val jobInfos = mutable.HashMap.empty[String, JobInfo] 41 | 42 | def saveJobInfo(jobInfo: JobInfo) { jobInfos(jobInfo.jobId) = jobInfo } 43 | 44 | def getJobInfos(): Map[String, JobInfo] = jobInfos.toMap 45 | 46 | override def getJobInfosLimit(limit: Int): Map[String, JobInfo] = jobInfos.takeRight(limit).toMap 47 | 48 | override def getJobInfo(jobId: String): Option[JobInfo] = jobInfos.get(jobId) 49 | 50 | val jobConfigs = mutable.HashMap.empty[String, Config] 51 | 52 | def saveJobConfig(jobId: String, jobConfig: Config) { jobConfigs(jobId) = jobConfig } 53 | 54 | def getJobConfigs(): Map[String, Config] = jobConfigs.toMap 55 | } 56 | -------------------------------------------------------------------------------- /job-server/test/spark.jobserver/JobInfoActorSpec.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.{Props, ActorRef, ActorSystem} 4 | import akka.testkit.{TestKit, ImplicitSender} 5 | import org.scalatest.{FunSpec, BeforeAndAfter, BeforeAndAfterAll} 6 | import org.scalatest.matchers.ShouldMatchers 7 | import spark.jobserver.io.JobDAO 8 | 9 | object JobInfoActorSpec { 10 | val system = ActorSystem("test") 11 | } 12 | 13 | class JobInfoActorSpec extends TestKit(JobInfoActorSpec.system) with ImplicitSender 14 | with FunSpec with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll { 15 | 16 | import com.typesafe.config._ 17 | import CommonMessages.NoSuchJobId 18 | import JobInfoActor._ 19 | 20 | private val jobId = "jobId" 21 | private val jobConfig = ConfigFactory.empty() 22 | 23 | override def afterAll() { 24 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(JobInfoActorSpec.system) 25 | } 26 | 27 | var actor: ActorRef = _ 28 | var dao: JobDAO = _ 29 | 30 | before { 31 | dao = new InMemoryDAO 32 | actor = system.actorOf(Props(classOf[JobInfoActor], dao, system.actorOf(Props(classOf[LocalContextSupervisorActor], dao)))) 33 | } 34 | 35 | after { 36 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(actor) 37 | } 38 | 39 | describe("JobInfoActor") { 40 | it("should store a job configuration") { 41 | actor ! StoreJobConfig(jobId, jobConfig) 42 | expectMsg(JobConfigStored) 43 | dao.getJobConfigs.get(jobId) should be (Some(jobConfig)) 44 | } 45 | 46 | it("should return a job configuration when the jobId exists") { 47 | actor ! StoreJobConfig(jobId, jobConfig) 48 | expectMsg(JobConfigStored) 49 | actor ! GetJobConfig(jobId) 50 | expectMsg(jobConfig) 51 | } 52 | 53 | it("should return error if jobId does not exist") { 54 | actor ! GetJobConfig(jobId) 55 | expectMsg(NoSuchJobId) 56 | } 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /job-server/test/spark.jobserver/JobManagerActorAdHocSpec.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.Props 4 | 5 | /** 6 | * This tests JobManagerActor of AdHoc context. 7 | * Pass true to isAdHoc when the JobManagerActor is created. 8 | */ 9 | class JobManagerActorAdHocSpec extends JobManagerSpec { 10 | 11 | before { 12 | dao = new InMemoryDAO 13 | manager = 14 | system.actorOf(JobManagerActor.props(dao, "test", JobManagerSpec.config, true)) 15 | } 16 | 17 | } 18 | -------------------------------------------------------------------------------- /job-server/test/spark.jobserver/JobManagerActorSpec.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.Props 4 | import com.codahale.metrics.Counter 5 | import ooyala.common.akka.metrics.MetricsWrapper 6 | import spark.jobserver.CommonMessages.{JobErroredOut, JobResult} 7 | 8 | class JobManagerActorSpec extends JobManagerSpec { 9 | import scala.concurrent.duration._ 10 | 11 | // Metrics for the job cache 12 | private val metricCacheHitName = "spark.jobserver.JobCache.cache-hit" 13 | private val metricCacheMissName = "spark.jobserver.JobCache.cache-miss" 14 | var metricCacheHit: Counter = _ 15 | var metricCacheMiss: Counter = _ 16 | 17 | before { 18 | // Resets the metrics for the job cache 19 | MetricsWrapper.getRegistry.remove(metricCacheHitName) 20 | MetricsWrapper.getRegistry.remove(metricCacheMissName) 21 | MetricsWrapper.getRegistry.counter(metricCacheHitName) 22 | MetricsWrapper.getRegistry.counter(metricCacheMissName) 23 | 24 | val counters = MetricsWrapper.getRegistry.getCounters 25 | metricCacheHit = counters.get(metricCacheHitName) 26 | metricCacheMiss = counters.get(metricCacheMissName) 27 | 28 | dao = new InMemoryDAO 29 | manager = 30 | system.actorOf(JobManagerActor.props(dao, "test", JobManagerSpec.config, false)) 31 | } 32 | 33 | describe("starting jobs") { 34 | it("jobs should be able to cache RDDs and retrieve them through getPersistentRDDs") { 35 | manager ! JobManagerActor.Initialize 36 | expectMsgClass(classOf[JobManagerActor.Initialized]) 37 | 38 | uploadTestJar() 39 | manager ! JobManagerActor.StartJob("demo", classPrefix + "CacheSomethingJob", emptyConfig, 40 | errorEvents ++ syncEvents) 41 | val JobResult(_, sum: Int) = expectMsgClass(classOf[JobResult]) 42 | 43 | manager ! JobManagerActor.StartJob("demo", classPrefix + "AccessCacheJob", emptyConfig, 44 | errorEvents ++ syncEvents) 45 | val JobResult(_, sum2: Int) = expectMsgClass(classOf[JobResult]) 46 | 47 | sum2 should equal (sum) 48 | } 49 | 50 | it ("jobs should be able to cache and retrieve RDDs by name") { 51 | manager ! JobManagerActor.Initialize 52 | expectMsgClass(classOf[JobManagerActor.Initialized]) 53 | 54 | uploadTestJar() 55 | manager ! JobManagerActor.StartJob("demo", classPrefix + "CacheRddByNameJob", emptyConfig, 56 | errorEvents ++ syncEvents) 57 | expectMsgPF(1 second, "Expected a JobResult or JobErroredOut message!") { 58 | case JobResult(_, sum: Int) => sum should equal (1 + 4 + 9 + 16 + 25) 59 | case JobErroredOut(_, _, error: Throwable) => throw error 60 | } 61 | } 62 | } 63 | 64 | describe("JobManagerActor JobCache") { 65 | it("should increase job cache hit count") { 66 | metricCacheHit.getCount should equal(0) 67 | metricCacheMiss.getCount should equal(0) 68 | 69 | manager ! JobManagerActor.Initialize 70 | expectMsgClass(classOf[JobManagerActor.Initialized]) 71 | 72 | uploadTestJar() 73 | 74 | manager ! JobManagerActor.StartJob("demo", classPrefix + "SimpleObjectJob", emptyConfig, 75 | errorEvents ++ syncEvents) 76 | expectMsgClass(classOf[JobResult]) 77 | metricCacheHit.getCount should equal(0) 78 | // Demo for the first time, misses the job cache 79 | metricCacheMiss.getCount should equal(1) 80 | 81 | manager ! JobManagerActor.StartJob("demo", classPrefix + "SimpleObjectJob", emptyConfig, 82 | errorEvents ++ syncEvents) 83 | expectMsgClass(classOf[JobResult]) 84 | // Demo for the second time, hits the job cache 85 | metricCacheHit.getCount should equal(1) 86 | metricCacheMiss.getCount should equal(1) 87 | } 88 | 89 | it("should increase job cache miss count") { 90 | metricCacheHit.getCount should equal(0) 91 | metricCacheMiss.getCount should equal(0) 92 | 93 | manager ! JobManagerActor.Initialize 94 | expectMsgClass(classOf[JobManagerActor.Initialized]) 95 | 96 | uploadTestJar() 97 | 98 | manager ! JobManagerActor.StartJob("demo", classPrefix + "SimpleObjectJob", emptyConfig, 99 | errorEvents ++ syncEvents) 100 | expectMsgClass(classOf[JobResult]) 101 | metricCacheHit.getCount should equal(0) 102 | // Demo for the first time, misses the job cache 103 | metricCacheMiss.getCount should equal(1) 104 | 105 | uploadTestJar("new demo") 106 | 107 | manager ! JobManagerActor.StartJob("new demo", classPrefix + "SimpleObjectJob", emptyConfig, 108 | errorEvents ++ syncEvents) 109 | expectMsgClass(classOf[JobResult]) 110 | metricCacheHit.getCount should equal(0) 111 | // New demo for the first time, misses the job cache again 112 | metricCacheMiss.getCount should equal(2) 113 | } 114 | } 115 | } 116 | -------------------------------------------------------------------------------- /job-server/test/spark.jobserver/JobResultActorSpec.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.{ActorRef, ActorSystem} 4 | import akka.testkit.{ImplicitSender, TestActorRef, TestKit} 5 | import org.scalatest.matchers.ShouldMatchers 6 | import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSpec} 7 | 8 | 9 | object JobResultActorSpec { 10 | val system = ActorSystem("job-result-actor-test") 11 | } 12 | 13 | class JobResultActorSpec extends TestKit(JobResultActorSpec.system) with ImplicitSender 14 | with FunSpec with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll { 15 | 16 | import CommonMessages._ 17 | 18 | override def afterAll() { 19 | TestKit.shutdownActorSystem(JobResultActorSpec.system) 20 | } 21 | 22 | describe("JobResultActor") { 23 | it("should return error if non-existing jobs are asked") { 24 | withActor { actor => 25 | actor ! GetJobResult("jobId") 26 | expectMsg(NoSuchJobId) 27 | } 28 | } 29 | 30 | it("should get back existing result") { 31 | withActor { actor => 32 | actor ! JobResult("jobId", 10) 33 | actor ! GetJobResult("jobId") 34 | expectMsg(JobResult("jobId", 10)) 35 | } 36 | } 37 | 38 | it("should be informed only once by subscribed result") { 39 | withActor { actor => 40 | actor ! Subscribe("jobId", self, Set(classOf[JobResult])) 41 | actor ! JobResult("jobId", 10) 42 | expectMsg(JobResult("jobId", 10)) 43 | 44 | actor ! JobResult("jobId", 20) 45 | expectNoMsg() // shouldn't get it again 46 | } 47 | } 48 | 49 | it("should not be informed unsubscribed result") { 50 | withActor { actor => 51 | actor ! Subscribe("jobId", self, Set(classOf[JobResult])) 52 | actor ! Unsubscribe("jobId", self) 53 | actor ! JobResult("jobId", 10) 54 | expectNoMsg() 55 | } 56 | } 57 | 58 | it("should not publish if do not subscribe to JobResult events") { 59 | withActor { actor => 60 | actor ! Subscribe("jobId", self, Set(classOf[JobValidationFailed], classOf[JobErroredOut])) 61 | actor ! JobResult("jobId", 10) 62 | expectNoMsg() 63 | } 64 | } 65 | 66 | it("should return error if non-existing subscription is unsubscribed") { 67 | withActor{ actor => 68 | actor ! Unsubscribe("jobId", self) 69 | expectMsg(NoSuchJobId) 70 | } 71 | } 72 | } 73 | 74 | def withActor[T](f: ActorRef => T): T ={ 75 | val actor = TestActorRef(new JobResultActor) 76 | try { 77 | f(actor) 78 | } finally { 79 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(actor) 80 | } 81 | } 82 | } -------------------------------------------------------------------------------- /job-server/test/spark.jobserver/JobResultCacheSpec.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.{ActorRef, ActorSystem, Props} 4 | import akka.testkit.{ImplicitSender, TestActorRef, TestKit} 5 | import com.typesafe.config.ConfigFactory 6 | import ooyala.common.akka.metrics.MetricsWrapper 7 | import org.scalatest.matchers.ShouldMatchers 8 | import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSpec} 9 | 10 | 11 | object JobResultCacheSpec { 12 | val config = ConfigFactory.parseString(""" 13 | spark { 14 | jobserver.job-result-cache-size = 10 15 | jobserver.job-result-cache-ttl-seconds = 1 16 | } 17 | """) 18 | 19 | val system = ActorSystem("job-result-cache-test", config) 20 | } 21 | 22 | class JobResultCacheSpec extends TestKit(JobResultCacheSpec.system) with ImplicitSender 23 | with FunSpec with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll { 24 | 25 | import CommonMessages._ 26 | 27 | override def afterAll() { 28 | TestKit.shutdownActorSystem(JobResultCacheSpec.system) 29 | } 30 | 31 | // Metrics for job result cache 32 | private val metricCacheHitName = "spark.jobserver.JobResultActor.cache-hit" 33 | private val metricCacheMissName = "spark.jobserver.JobResultActor.cache-miss" 34 | private val metricCacheRequestName = "spark.jobserver.JobResultActor.cache-request" 35 | private val metricCacheEvictionName = "spark.jobserver.JobResultActor.cache-eviction" 36 | 37 | before { 38 | MetricsWrapper.getRegistry.remove(metricCacheHitName) 39 | MetricsWrapper.getRegistry.remove(metricCacheMissName) 40 | MetricsWrapper.getRegistry.remove(metricCacheRequestName) 41 | MetricsWrapper.getRegistry.remove(metricCacheEvictionName) 42 | } 43 | 44 | describe("JobResultActor cache") { 45 | it("should increase cache hit count") { 46 | withActor { actor => 47 | 48 | actor ! JobResult("jobId", 10) 49 | actor ! GetJobResult("jobId") 50 | expectMsg(JobResult("jobId", 10)) 51 | // Hits the job result cache for the first time 52 | MetricsWrapper.getRegistry.getGauges.get(metricCacheHitName).getValue should equal(1) 53 | MetricsWrapper.getRegistry.getGauges.get(metricCacheMissName).getValue should equal(0) 54 | MetricsWrapper.getRegistry.getGauges.get(metricCacheRequestName).getValue should equal(1) 55 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(0) 56 | 57 | actor ! GetJobResult("jobId") 58 | expectMsg(JobResult("jobId", 10)) 59 | // Hits the job result cache for the second time 60 | MetricsWrapper.getRegistry.getGauges.get(metricCacheHitName).getValue should equal(2) 61 | MetricsWrapper.getRegistry.getGauges.get(metricCacheMissName).getValue should equal(0) 62 | MetricsWrapper.getRegistry.getGauges.get(metricCacheRequestName).getValue should equal(2) 63 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(0) 64 | } 65 | } 66 | 67 | it("should increase cache miss count") { 68 | withActor { actor => 69 | actor ! JobResult("jobId", 10) 70 | actor ! GetJobResult("NoJobId") 71 | expectMsg(NoSuchJobId) 72 | MetricsWrapper.getRegistry.getGauges.get(metricCacheHitName).getValue should equal(0) 73 | MetricsWrapper.getRegistry.getGauges.get(metricCacheMissName).getValue should equal(1) 74 | MetricsWrapper.getRegistry.getGauges.get(metricCacheRequestName).getValue should equal(1) 75 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(0) 76 | 77 | actor ! GetJobResult("NoJobId") 78 | expectMsg(NoSuchJobId) 79 | MetricsWrapper.getRegistry.getGauges.get(metricCacheHitName).getValue should equal(0) 80 | MetricsWrapper.getRegistry.getGauges.get(metricCacheMissName).getValue should equal(2) 81 | MetricsWrapper.getRegistry.getGauges.get(metricCacheRequestName).getValue should equal(2) 82 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(0) 83 | } 84 | } 85 | 86 | it("should evict entries according to configured TTL") { 87 | withActor { actor => 88 | val jobResultActor = system.actorOf(Props[JobResultActor]) 89 | 90 | (1 to 10).foreach { i => 91 | actor ! JobResult(s"jobId_$i", 10) 92 | } 93 | 94 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(0) 95 | 96 | //from javadoc: requested entries may be evicted on each cache modification, on occasional 97 | //cache accesses, or on calls to Cache#cleanUp 98 | Thread.sleep(1500) 99 | actor ! JobResult("jobId", 10) 100 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(10) 101 | actor ! GetJobResult("jobId_1") 102 | MetricsWrapper.getRegistry.getGauges.get(metricCacheMissName).getValue should equal(1) 103 | } 104 | } 105 | 106 | it("should evict entries according to configured cache size") { 107 | withActor { actor => 108 | (1 to 10).foreach { i => 109 | actor ! JobResult(s"jobId_$i", 10) 110 | } 111 | 112 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(0) 113 | 114 | actor ! JobResult("jobId_X", 10) 115 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(1) 116 | 117 | actor ! JobResult("jobId_Y", 10) 118 | MetricsWrapper.getRegistry.getGauges.get(metricCacheEvictionName).getValue should equal(2) 119 | } 120 | } 121 | } 122 | 123 | def withActor[T](f: ActorRef => T): T ={ 124 | val actor = TestActorRef(new JobResultActor) 125 | try { 126 | f(actor) 127 | } finally { 128 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(actor) 129 | } 130 | } 131 | } 132 | -------------------------------------------------------------------------------- /job-server/test/spark.jobserver/JobStatusActorSpec.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.{Props, PoisonPill, ActorRef, ActorSystem} 4 | import akka.testkit.{TestKit, ImplicitSender} 5 | import spark.jobserver.io.{JarInfo, JobInfo, JobDAO} 6 | import org.joda.time.DateTime 7 | import org.scalatest.matchers.ShouldMatchers 8 | import org.scalatest.{FunSpec, BeforeAndAfter, BeforeAndAfterAll} 9 | 10 | object JobStatusActorSpec { 11 | val system = ActorSystem("test") 12 | } 13 | 14 | class JobStatusActorSpec extends TestKit(JobStatusActorSpec.system) with ImplicitSender 15 | with FunSpec with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll { 16 | 17 | import com.typesafe.config._ 18 | import CommonMessages._ 19 | import JobStatusActor._ 20 | 21 | private val jobId = "jobId" 22 | private val contextName = "contextName" 23 | private val appName = "appName" 24 | private val jarInfo = JarInfo(appName, DateTime.now) 25 | private val classPath = "classPath" 26 | private val jobInfo = JobInfo(jobId, contextName, jarInfo, classPath, DateTime.now, None, None) 27 | private val jobConfig = ConfigFactory.empty() 28 | 29 | override def afterAll() { 30 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(JobStatusActorSpec.system) 31 | } 32 | 33 | var actor: ActorRef = _ 34 | var receiver: ActorRef = _ 35 | var dao: JobDAO = _ 36 | 37 | before { 38 | dao = new InMemoryDAO 39 | actor = system.actorOf(Props(classOf[JobStatusActor], dao)) 40 | receiver = system.actorOf(Props[JobResultActor]) 41 | } 42 | 43 | after { 44 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(actor) 45 | } 46 | 47 | describe("JobStatusActor") { 48 | it("should return empty sequence if there is no job infos") { 49 | actor ! GetRunningJobStatus 50 | expectMsg(Seq.empty) 51 | } 52 | 53 | it("should return error if non-existing job is unsubscribed") { 54 | actor ! Unsubscribe(jobId, receiver) 55 | expectMsg(NoSuchJobId) 56 | } 57 | 58 | it("should not initialize a job more than two times") { 59 | actor ! JobInit(jobInfo) 60 | actor ! JobInit(jobInfo) 61 | expectMsg(JobInitAlready) 62 | } 63 | 64 | it("should be informed JobStarted until it is unsubscribed") { 65 | actor ! JobInit(jobInfo) 66 | actor ! Subscribe(jobId, self, Set(classOf[JobStarted])) 67 | val msg = JobStarted(jobId, contextName, DateTime.now) 68 | actor ! msg 69 | expectMsg(msg) 70 | 71 | actor ! msg 72 | expectMsg(msg) 73 | 74 | actor ! Unsubscribe(jobId, self) 75 | actor ! JobStarted(jobId, contextName, DateTime.now) 76 | expectNoMsg() // shouldn't get it again 77 | 78 | actor ! Unsubscribe(jobId, self) 79 | expectMsg(NoSuchJobId) 80 | } 81 | 82 | it("should be ok to subscribe beofore job init") { 83 | actor ! Subscribe(jobId, self, Set(classOf[JobStarted])) 84 | actor ! JobInit(jobInfo) 85 | val msg = JobStarted(jobId, contextName, DateTime.now) 86 | actor ! msg 87 | expectMsg(msg) 88 | } 89 | 90 | it("should be informed JobValidationFailed once") { 91 | actor ! JobInit(jobInfo) 92 | actor ! Subscribe(jobId, self, Set(classOf[JobValidationFailed])) 93 | val msg = JobValidationFailed(jobId, DateTime.now, new Throwable) 94 | actor ! msg 95 | expectMsg(msg) 96 | 97 | actor ! msg 98 | expectMsg(NoSuchJobId) 99 | } 100 | 101 | it("should be informed JobFinished until it is unsubscribed") { 102 | actor ! JobInit(jobInfo) 103 | actor ! JobStarted(jobId, contextName, DateTime.now) 104 | actor ! Subscribe(jobId, self, Set(classOf[JobFinished])) 105 | val msg = JobFinished(jobId, DateTime.now) 106 | actor ! msg 107 | expectMsg(msg) 108 | 109 | actor ! msg 110 | expectMsg(NoSuchJobId) 111 | } 112 | 113 | it("should be informed JobErroredOut until it is unsubscribed") { 114 | actor ! JobInit(jobInfo) 115 | actor ! JobStarted(jobId, contextName, DateTime.now) 116 | actor ! Subscribe(jobId, self, Set(classOf[JobErroredOut])) 117 | val msg = JobErroredOut(jobId, DateTime.now, new Throwable) 118 | actor ! msg 119 | expectMsg(msg) 120 | 121 | actor ! msg 122 | expectMsg(NoSuchJobId) 123 | } 124 | 125 | it("should update status correctly") { 126 | actor ! JobInit(jobInfo) 127 | actor ! GetRunningJobStatus 128 | expectMsg(Seq(jobInfo)) 129 | 130 | val startTime = DateTime.now 131 | actor ! JobStarted(jobId, contextName, startTime) 132 | actor ! GetRunningJobStatus 133 | expectMsg(Seq(JobInfo(jobId, contextName, jarInfo, classPath, startTime, None, None))) 134 | 135 | val finishTIme = DateTime.now 136 | actor ! JobFinished(jobId, finishTIme) 137 | actor ! GetRunningJobStatus 138 | expectMsg(Seq.empty) 139 | } 140 | 141 | it("should update JobValidationFailed status correctly") { 142 | val initTime = DateTime.now 143 | val jobInfo = JobInfo(jobId, contextName, jarInfo, classPath, initTime, None, None) 144 | actor ! JobInit(jobInfo) 145 | 146 | val failedTime = DateTime.now 147 | val err = new Throwable 148 | actor ! JobValidationFailed(jobId, failedTime, err) 149 | actor ! GetRunningJobStatus 150 | expectMsg(Seq.empty) 151 | } 152 | 153 | it("should update JobErroredOut status correctly") { 154 | actor ! JobInit(jobInfo) 155 | 156 | val startTime = DateTime.now 157 | actor ! JobStarted(jobId, contextName, startTime) 158 | 159 | val failedTime = DateTime.now 160 | val err = new Throwable 161 | actor ! JobErroredOut(jobId, failedTime, err) 162 | actor ! GetRunningJobStatus 163 | expectMsg(Seq.empty) 164 | } 165 | } 166 | } 167 | -------------------------------------------------------------------------------- /job-server/test/spark.jobserver/LocalContextSupervisorHaSpec.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.{ActorRef, Props, ActorSystem} 4 | import akka.testkit.{ImplicitSender, TestKit} 5 | import akka.util.Timeout 6 | import com.typesafe.config.ConfigFactory 7 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSpec} 8 | import org.scalatest.matchers.ShouldMatchers 9 | import scala.concurrent.duration._ 10 | import spark.jobserver.io.JobSqlDAO 11 | 12 | 13 | class LocalContextSupervisorHaSpec(system: ActorSystem) extends TestKit(system) with ImplicitSender 14 | with FunSpec with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll { 15 | 16 | def this() = this(ActorSystem("test", LocalContextSupervisorSpec.config)) 17 | 18 | override def afterAll() { 19 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(system) 20 | } 21 | 22 | // Expects the response to GetContext message for the named context. 23 | private def expectResponseToGetContext(contextName: String) = { 24 | expectMsgPF() { 25 | case (manager: ActorRef, resultActor: ActorRef) => 26 | manager.path.name should equal(contextName) 27 | resultActor.path.name should equal("result-actor") 28 | } 29 | } 30 | 31 | var supervisor_a: ActorRef = _ 32 | var supervisor_b: ActorRef = _ 33 | val contextConfig = LocalContextSupervisorSpec.config.getConfig("spark.context-settings") 34 | 35 | implicit val timeout: Timeout = 10 seconds 36 | 37 | // This is needed to help tests pass on some MBPs when working from home 38 | System.setProperty("spark.driver.host", "localhost") 39 | 40 | before { 41 | val sqlDaoConfigStr = "spark.jobserver.sqldao.h2.url=\"jdbc:h2:file:" + 42 | "/tmp/spark-job-server-test/sqldao/data/" + java.util.UUID.randomUUID() + "\"" 43 | val sqlDaoConfig = ConfigFactory.parseString(sqlDaoConfigStr) 44 | 45 | val dao_a = new JobSqlDAO(sqlDaoConfig) 46 | supervisor_a = system.actorOf(Props(classOf[LocalContextSupervisorActor], dao_a)) 47 | 48 | val dao_b = new JobSqlDAO(sqlDaoConfig) 49 | supervisor_b = system.actorOf(Props(classOf[LocalContextSupervisorActor], dao_b)) 50 | } 51 | 52 | after { 53 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(supervisor_a) 54 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(supervisor_b) 55 | } 56 | 57 | import ContextSupervisor._ 58 | 59 | describe("context high availability") { 60 | it("should return no such context if DAO hasn't added it") { 61 | supervisor_a ! AddContext("c1", contextConfig) 62 | expectMsg(ContextInitialized) 63 | 64 | supervisor_b ! GetContext("c2") 65 | expectMsg(NoSuchContext) 66 | } 67 | 68 | it("should allow two context supervisors to share the same context config") { 69 | supervisor_a ! AddContext("c1", contextConfig) 70 | expectMsg(ContextInitialized) 71 | 72 | supervisor_b ! GetContext("c1") 73 | expectResponseToGetContext("c1") 74 | } 75 | 76 | it("should return ContextJobDaoError if context config already added by DAO") { 77 | supervisor_a ! AddContext("c1", contextConfig) 78 | expectMsg(ContextInitialized) 79 | 80 | supervisor_b ! AddContext("c1", contextConfig) 81 | expectMsgType[ContextJobDaoError] 82 | } 83 | 84 | it("should retrieve context config from persistent storage after context stopped") { 85 | supervisor_a ! AddContext("c1", contextConfig) 86 | expectMsg(ContextInitialized) 87 | 88 | supervisor_b ! GetContext("c1") 89 | expectResponseToGetContext("c1") 90 | 91 | supervisor_b ! StopContext("c1") 92 | expectMsg(ContextStopped) 93 | 94 | Thread.sleep(2000) // wait for a while since deleting context is an asyc call 95 | 96 | supervisor_b ! GetContext("c1") 97 | expectResponseToGetContext("c1") 98 | } 99 | } 100 | } -------------------------------------------------------------------------------- /job-server/test/spark.jobserver/LocalContextSupervisorSpec.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor._ 4 | import akka.testkit.{TestKit, ImplicitSender} 5 | import com.typesafe.config.ConfigFactory 6 | import ooyala.common.akka.metrics.MetricsWrapper 7 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter} 8 | import org.scalatest.FunSpec 9 | import org.scalatest.matchers.ShouldMatchers 10 | import spark.jobserver.CommonMessages.{Unsubscribe, JobStarted, Subscribe} 11 | import spark.jobserver.io.JobDAO 12 | 13 | object LocalContextSupervisorSpec { 14 | val config = ConfigFactory.parseString(""" 15 | spark { 16 | master = "local[4]" 17 | temp-contexts { 18 | num-cpu-cores = 4 # Number of cores to allocate. Required. 19 | memory-per-node = 512m # Executor memory per node, -Xmx style eg 512m, 1G, etc. 20 | } 21 | jobserver.job-result-cache-size = 100 22 | jobserver.job-result-cache-ttl-seconds = 60 23 | jobserver.context-creation-timeout = 5 s 24 | jobserver.context-factory = spark.jobserver.util.DefaultSparkContextFactory 25 | contexts { 26 | olap-demo { 27 | num-cpu-cores = 4 28 | memory-per-node = 512m 29 | } 30 | } 31 | context-settings { 32 | num-cpu-cores = 2 33 | memory-per-node = 512m 34 | } 35 | } 36 | akka.log-dead-letters = 0 37 | """) 38 | 39 | val system = ActorSystem("test", config) 40 | } 41 | 42 | class LocalContextSupervisorSpec extends TestKit(LocalContextSupervisorSpec.system) with ImplicitSender 43 | with FunSpec with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll { 44 | 45 | override def afterAll() { 46 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(LocalContextSupervisorSpec.system) 47 | } 48 | 49 | var supervisor: ActorRef = _ 50 | var dao: JobDAO = _ 51 | 52 | val contextConfig = LocalContextSupervisorSpec.config.getConfig("spark.context-settings") 53 | 54 | val counterName = "spark.jobserver.LocalContextSupervisorActor.num-contexts" 55 | 56 | // This is needed to help tests pass on some MBPs when working from home 57 | System.setProperty("spark.driver.host", "localhost") 58 | 59 | before { 60 | dao = new InMemoryDAO 61 | supervisor = system.actorOf(Props(classOf[LocalContextSupervisorActor], dao)) 62 | } 63 | 64 | after { 65 | val counters: java.util.SortedMap[String, com.codahale.metrics.Counter] = MetricsWrapper.registry.getCounters 66 | val counter = counters.get(counterName) 67 | counter.dec(counter.getCount) 68 | 69 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(supervisor) 70 | 71 | } 72 | 73 | import ContextSupervisor._ 74 | 75 | describe("context management") { 76 | it("should list empty contexts at startup") { 77 | supervisor ! ListContexts 78 | expectMsg(Seq.empty[String]) 79 | } 80 | 81 | it("can add contexts from jobConfig") { 82 | supervisor ! AddContextsFromConfig 83 | Thread sleep 2000 84 | supervisor ! ListContexts 85 | expectMsg(Seq("olap-demo")) 86 | } 87 | 88 | it("should be able to add multiple new contexts") { 89 | supervisor ! AddContext("c1", contextConfig) 90 | supervisor ! AddContext("c2", contextConfig) 91 | expectMsg(ContextInitialized) 92 | expectMsg(ContextInitialized) 93 | supervisor ! ListContexts 94 | expectMsg(Seq("c1", "c2")) 95 | supervisor ! GetResultActor("c1") 96 | val rActor = expectMsgClass(classOf[ActorRef]) 97 | rActor.path.toString should endWith ("result-actor") 98 | rActor.path.toString should not include ("global") 99 | } 100 | 101 | it("should be able to stop contexts already running") { 102 | import scala.concurrent.duration._ 103 | supervisor ! AddContext("c1", contextConfig) 104 | expectMsg(ContextInitialized) 105 | supervisor ! ListContexts 106 | expectMsg(Seq("c1")) 107 | 108 | supervisor ! StopContext("c1") 109 | expectMsg(ContextStopped) 110 | 111 | Thread.sleep(2000) // wait for a while since deleting context is an asyc call 112 | supervisor ! ListContexts 113 | expectMsg(Seq.empty[String]) 114 | } 115 | 116 | it("should return NoSuchContext if attempt to stop nonexisting context") { 117 | supervisor ! StopContext("c1") 118 | expectMsg(NoSuchContext) 119 | } 120 | 121 | it("should not allow creation of an already existing context") { 122 | supervisor ! AddContext("c1", contextConfig) 123 | expectMsg(ContextInitialized) 124 | 125 | supervisor ! AddContext("c1", contextConfig) 126 | expectMsg(ContextAlreadyExists) 127 | } 128 | 129 | it("should inc/dec context counter correctly") { 130 | val counters = MetricsWrapper.registry.getCounters 131 | val counter = counters.get(counterName) 132 | counter should not be (null) 133 | 134 | supervisor ! AddContext("c11", contextConfig) 135 | expectMsg(ContextInitialized) 136 | supervisor ! ListContexts 137 | expectMsg(Seq("c11")) 138 | 139 | (counter.getCount) should equal (1) 140 | 141 | supervisor ! StopContext("c11") 142 | expectMsg(ContextStopped) 143 | supervisor ! ListContexts 144 | expectMsg(Seq.empty[String]) 145 | 146 | 147 | (counter.getCount) should equal (0) 148 | } 149 | 150 | 151 | } 152 | } 153 | -------------------------------------------------------------------------------- /job-server/test/spark.jobserver/NamedRddsSpec.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.{PoisonPill, Props, ActorRef, ActorSystem} 4 | import akka.testkit.{ImplicitSender, TestKit} 5 | import org.apache.spark.SparkContext 6 | import org.scalatest.matchers.ShouldMatchers 7 | import org.scalatest.{FunSpec, BeforeAndAfterAll, BeforeAndAfter} 8 | 9 | class NamedRddsSpec extends TestKit(ActorSystem("NamedRddsSpec")) with FunSpec 10 | with ImplicitSender with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll { 11 | System.setProperty("spark.cores.max", Runtime.getRuntime.availableProcessors.toString) 12 | System.setProperty("spark.executor.memory", "512m") 13 | System.setProperty("spark.akka.threads", Runtime.getRuntime.availableProcessors.toString) 14 | 15 | // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown 16 | System.clearProperty("spark.driver.port") 17 | System.clearProperty("spark.hostPort") 18 | 19 | val sc = new SparkContext("local[4]", getClass.getSimpleName) 20 | val rddManager: ActorRef = system.actorOf(Props(classOf[RddManagerActor], sc)) 21 | val namedRdds: NamedRdds = new JobServerNamedRdds(rddManager) 22 | 23 | before { 24 | namedRdds.getNames.foreach { rddName => namedRdds.destroy(rddName) } 25 | } 26 | 27 | override def afterAll() { 28 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(rddManager) 29 | sc.stop() 30 | ooyala.common.akka.AkkaTestUtils.shutdownAndWait(system) 31 | } 32 | 33 | describe("NamedRdds") { 34 | it("get() should return None when RDD does not exist") { 35 | namedRdds.get[Int]("No such RDD") should equal (None) 36 | } 37 | 38 | it("get() should return Some(RDD) when it exists") { 39 | val rdd = sc.parallelize(Seq(1, 2, 3)) 40 | namedRdds.update("rdd1", rdd) 41 | namedRdds.get[Int]("rdd1") should equal (Some(rdd)) 42 | } 43 | 44 | it("destroy() should do nothing when RDD with given name doesn't exist") { 45 | namedRdds.update("rdd1", sc.parallelize(Seq(1, 2, 3))) 46 | namedRdds.get[Int]("rdd1") should not equal None 47 | namedRdds.destroy("rdd2") 48 | namedRdds.get[Int]("rdd1") should not equal None 49 | } 50 | 51 | it("destroy() should destroy an RDD that exists") { 52 | namedRdds.update("rdd1", sc.parallelize(Seq(1, 2, 3))) 53 | namedRdds.get[Int]("rdd1") should not equal None 54 | namedRdds.destroy("rdd1") 55 | namedRdds.get[Int]("rdd1") should equal (None) 56 | } 57 | 58 | it("getNames() should return names of all managed RDDs") { 59 | namedRdds.getNames() should equal (Iterable()) 60 | namedRdds.update("rdd1", sc.parallelize(Seq(1, 2, 3))) 61 | namedRdds.update("rdd2", sc.parallelize(Seq(4, 5, 6))) 62 | namedRdds.getNames().toSeq.sorted should equal (Seq("rdd1", "rdd2")) 63 | namedRdds.destroy("rdd1") 64 | namedRdds.getNames().toSeq.sorted should equal (Seq("rdd2")) 65 | } 66 | 67 | it("getOrElseCreate() should call generator function if RDD does not exist") { 68 | var generatorCalled = false 69 | val rdd = namedRdds.getOrElseCreate("rdd1", { 70 | generatorCalled = true 71 | sc.parallelize(Seq(1, 2, 3)) 72 | }) 73 | generatorCalled should equal (true) 74 | } 75 | 76 | it("getOrElseCreate() should not call generator function, should return existing RDD if one exists") { 77 | var generatorCalled = false 78 | val rdd = sc.parallelize(Seq(1, 2, 3)) 79 | namedRdds.update("rdd1", rdd) 80 | val rdd2 = namedRdds.getOrElseCreate("rdd1", { 81 | generatorCalled = true 82 | sc.parallelize(Seq(4, 5, 6)) 83 | }) 84 | generatorCalled should equal (false) 85 | rdd2 should equal (rdd) 86 | } 87 | 88 | it("update() should replace existing RDD") { 89 | val rdd1 = sc.parallelize(Seq(1, 2, 3)) 90 | val rdd2 = sc.parallelize(Seq(4, 5, 6)) 91 | namedRdds.getOrElseCreate("rdd", rdd1) should equal (rdd1) 92 | namedRdds.update("rdd", rdd2) 93 | namedRdds.get[Int]("rdd") should equal (Some(rdd2)) 94 | } 95 | 96 | it("should include underlying exception when error occurs") { 97 | def errorFunc = { 98 | throw new IllegalArgumentException("boo!") 99 | sc.parallelize(Seq(1, 2)) 100 | } 101 | val err = intercept[RuntimeException] { namedRdds.getOrElseCreate("rdd", errorFunc) } 102 | err.getCause.getClass should equal (classOf[IllegalArgumentException]) 103 | } 104 | // TODO: Add tests for parallel getOrElseCreate() calls 105 | } 106 | } 107 | -------------------------------------------------------------------------------- /job-server/test/spark.jobserver/SparkJobSpec.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import org.scalatest.FunSpec 4 | import org.scalatest.matchers.ShouldMatchers 5 | 6 | class SparkJobSpec extends FunSpec with ShouldMatchers{ 7 | val validSparkValidation = SparkJobValid 8 | val invalidSparkValidation = SparkJobInvalid("Sample message 1") 9 | val invalidSparkValidation2 = SparkJobInvalid("Sample message 2") 10 | 11 | describe("Sample tests for default validation && method") { 12 | it("should return valid") { 13 | validSparkValidation && (validSparkValidation) should equal (SparkJobValid) 14 | } 15 | 16 | it("should return invalid if one of them is invalid") { 17 | validSparkValidation && (invalidSparkValidation) should equal (SparkJobInvalid("Sample message 1")) 18 | } 19 | 20 | it ("should return invalid if both of them are invalid with the first message") { 21 | invalidSparkValidation2 && (invalidSparkValidation) should equal (SparkJobInvalid("Sample message 2")) 22 | } 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /job-server/test/spark.jobserver/SparkWebUiActorSpec.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | import akka.actor.{Props, ActorRef, ActorSystem} 4 | import akka.io.IO 5 | import akka.pattern.ask 6 | import akka.testkit.{TestKit, ImplicitSender} 7 | import com.typesafe.config.ConfigFactory 8 | import org.scalatest.{FunSpec, BeforeAndAfter, BeforeAndAfterAll} 9 | import org.scalatest.matchers.ShouldMatchers 10 | import spray.client.pipelining._ 11 | 12 | import scala.concurrent.{Await, Future} 13 | import scala.concurrent.duration._ 14 | import akka.util.Timeout 15 | import akka.actor._ 16 | import spray.can.Http 17 | import spray.http._ 18 | import HttpMethods._ 19 | import SparkWebUiActor._ 20 | 21 | import scala.util.{Failure, Success} 22 | 23 | object SparkWebUiActorSpec { 24 | val sparkWebUrl = "localhost" 25 | val sparkWebPort = 8098 26 | val config = ConfigFactory.parseString(s""" 27 | spark { 28 | master = "spark://badhost:7077,localhost:7077" 29 | webUrlPort = $sparkWebPort 30 | temp-contexts { 31 | num-cpu-cores = 4 # Number of cores to allocate. Required. 32 | memory-per-node = 512m # Executor memory per node, -Xmx style eg 512m, 1G, etc. 33 | } 34 | jobserver.job-result-cache-size = 100 35 | jobserver.job-result-cache-ttl-seconds = 60 36 | jobserver.context-creation-timeout = 5 s 37 | jobserver.context-factory = spark.jobserver.util.DefaultSparkContextFactory 38 | contexts { 39 | olap-demo { 40 | num-cpu-cores = 4 41 | memory-per-node = 512m 42 | } 43 | } 44 | context-settings { 45 | num-cpu-cores = 2 46 | memory-per-node = 512m 47 | } 48 | } 49 | akka.log-dead-letters = 0 50 | """) 51 | 52 | val system = ActorSystem("test", config) 53 | } 54 | 55 | // simple http service 56 | class SimpleHttpServer extends Actor with ActorLogging { 57 | implicit val timeout: Timeout = 1.second // for the actor 'asks' 58 | import context.dispatcher // ExecutionContext for the futures and scheduler 59 | 60 | def receive = { 61 | // when a new connection comes in we register ourselves as the connection handler 62 | case _: Http.Connected => sender ! Http.Register(self) 63 | case HttpRequest(GET, Uri.Path("/"), _, _, _) => { 64 | sender ! HttpResponse(entity = 65 | """blah blah 66 | |
  • 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/TestJarFinder.scala: -------------------------------------------------------------------------------- 1 | package spark.jobserver 2 | 3 | trait TestJarFinder { 4 | val testJarDir = "job-server-tests/target/" 5 | val testJar = { 6 | val allJars = new java.io.File(testJarDir).listFiles.toSeq.filter { file => 7 | val path = file.toString 8 | path.endsWith(".jar") && !path.endsWith("-tests.jar") && !path.endsWith("-sources.jar") && 9 | !path.endsWith("-javadoc.jar") && !path.contains("scoverage") 10 | } 11 | assert(allJars.size == 1, allJars.toList.toString) 12 | allJars.head 13 | } 14 | } -------------------------------------------------------------------------------- /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 | -------------------------------------------------------------------------------- /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/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/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/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 | -------------------------------------------------------------------------------- /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 | } -------------------------------------------------------------------------------- /project/Assembly.scala: -------------------------------------------------------------------------------- 1 | import sbt._ 2 | import Keys._ 3 | import sbtassembly.Plugin._ 4 | import AssemblyKeys._ 5 | 6 | object Assembly { 7 | lazy val settings = assemblySettings ++ Seq( 8 | jarName in assembly := "spark-job-server.jar", 9 | excludedJars in assembly <<= (fullClasspath in assembly) map { _ filter { cp => 10 | List("servlet-api", "guice-all", "junit", "uuid", 11 | "jetty", "jsp-api-2.0", "antlr", "avro", "slf4j-log4j", "log4j-1.2", 12 | "scala-actors", "spark", "commons-cli", "stax-api", "mockito").exists(cp.data.getName.startsWith(_)) 13 | } }, 14 | assembleArtifact in packageScala := false, // We don't need the Scala library, Spark already includes it 15 | mergeStrategy in assembly := { 16 | case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard 17 | case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard 18 | case "reference.conf" => MergeStrategy.concat 19 | case _ => MergeStrategy.first 20 | } 21 | ) 22 | } -------------------------------------------------------------------------------- /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 | -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=0.13.8 2 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.4") 2 | 3 | addSbtPlugin("io.spray" % "sbt-revolver" % "0.7.2") // For quick restarts for web development 4 | 5 | addSbtPlugin("com.typesafe.sbt" % "sbt-scalariform" % "1.3.0") 6 | 7 | addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.11.2") 8 | 9 | addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.3.2") 10 | 11 | resolvers += Classpaths.sbtPluginReleases 12 | 13 | addSbtPlugin("org.scoverage" %% "sbt-scoverage" % "0.98.0") 14 | 15 | resolvers += Resolver.url( 16 | "bintray-sbt-plugin-releases", 17 | url("http://dl.bintray.com/content/sbt/sbt-plugin-releases"))( 18 | Resolver.ivyStylePatterns) 19 | 20 | addSbtPlugin("me.lessis" % "bintray-sbt" % "0.1.1") 21 | 22 | addSbtPlugin("com.typesafe.sbt" % "sbt-native-packager" % "0.6.3") 23 | 24 | resolvers += "jgit-repo" at "http://download.eclipse.org/jgit/maven" 25 | 26 | addSbtPlugin("com.typesafe.sbt" % "sbt-git" % "0.6.4") 27 | --------------------------------------------------------------------------------