SlideShare ist ein Scribd-Unternehmen logo
1 von 72
Downloaden Sie, um offline zu lesen
Productionizing Spark

and the Spark REST Job Server
Evan Chan
Distinguished Engineer
@TupleJump
Who am I
• Distinguished Engineer, Tuplejump
• @evanfchan
• http://github.com/velvia
• User and contributor to Spark since 0.9
• Co-creator and maintainer of Spark Job Server
2
TupleJump
✤ Tuplejump is a big data technology leader providing solutions and
development partnership.
✤ FiloDB - Spark-based analytics database for time series and event
data (github.com/tuplejump/FiloDB)
✤ Calliope - the first Spark-Cassandra integration
✤ Stargate - an open source Lucene indexer for Cassandra
✤ SnackFS - open source HDFS for Cassandra
3
TupleJump - Big Data Dev Partners 4
Deploying Spark
5
Choices, choices, choices
• YARN, Mesos, Standalone?
• With a distribution?
• What environment?
• How should I deploy?
• Hosted options?
• What about dependencies?
6
BasicTerminology
• The Spark documentation is really quite good.
7
What all the clusters have in common
• YARN, Mesos, and Standalone all support the following
features:
–Running the Spark driver app in cluster mode
–Restarts of the driver app upon failure
–UI to examine state of workers and apps
8
Spark Standalone Mode
• The easiest clustering mode to deploy**
–Use make-distribution.sh to package, copy to all nodes
–sbin/start-master.sh on master node, then start slaves
–Test with spark-shell
• HA Master through Zookeeper election
• Must dedicate whole cluster to Spark
• In latest survey, used by almost half of Spark users
9
Apache Mesos
• Was started by Matias in 2007 before he worked on Spark!
• Can run your entire company on Mesos, not just big data
–Great support for micro services - Docker, Marathon
–Can run non-JVM workloads like MPI
• Commercial backing from Mesosphere
• Heavily used at Twitter and AirBNB
• The Mesosphere DCOS will revolutionize Spark et al deployment - “dcos package
install spark” !!
10
Mesos vsYARN
• Mesos is a two-level resource manager, with pluggable schedulers
–You can run YARN on Mesos, with YARN delegating resource offers
to Mesos (Project Myriad)
–You can run multiple schedulers within Mesos, and write your own
• If you’re already a Hadoop / Cloudera etc shop, YARN is easy choice
• If you’re starting out, go 100% Mesos
11
Mesos Coarse vs Fine-Grained
• Spark offers two modes to run Mesos Spark apps in (and you can
choose per driver app):
–coarse-grained: Spark allocates fixed number of workers for
duration of driver app
–fine-grained (default): Dynamic executor allocation per task,
but higher overhead per task
• Use coarse-grained if you run low-latency jobs
12
What about Datastax DSE?
• Cassandra, Hadoop, Spark all bundled in one distribution, collocated
• Custom cluster manager and HA/failover logic for Spark Master,
using Cassandra gossip
• Can use CFS (Cassandra-based HDFS), SnackFS, or plain Cassandra
tables for storage
–or use Tachyon to cache, then no need to collocate (use Mesosphere
DCOS)
13
Hosted Apache Spark
• Spark on Amazon EMR - first class citizen now
–Direct S3 access!
• Google Compute Engine - “Click to Deploy” Hadoop+Spark
• Databricks Cloud
• Many more coming
• What you notice about the different environments:
–Everybody has their own way of starting: spark-submit vs dse spark vs aws
emr … vs dcos spark …
14
Mesosphere DCOS
• Automates deployment to AWS,
Google, etc.
• Common API and UI, better cost
and control, cloud
• Load balancing and routing,
Mesos for resource sharing
• dcos package install spark
15
Configuring Spark
16
Building Spark
• Make sure you build for the right Hadoop version
• eg mvn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package
• Make sure you build for the right Scala version - Spark supports
both 2.10 and 2.11
17
Jars schmars
• Dependency conflicts are the worst part of Spark dev
• Every distro has slightly different jars - eg CDH < 5.4 packaged a different version of
Akka
• Leave out Hive if you don’t need it
• Use the Spark UI “Environment” tab to check jars and how they got there
• spark-submit —jars / —packages forwards jars to every executor (unless it’s an
HDFS / HTTP path)
• Spark-env.sh SPARK_CLASSPATH - include dep jars you’ve deployed to every node
18
Jars schmars
• You don’t need to package every dependency with your Spark application!
• spark-streaming is included in the distribution
• spark-streaming includes some Kafka jars already
• etc.
19
ClassPath Configuration Options
• spark.driver.userClassPathFirst, spark.executor.userClassPathFirst
• One way to solve dependency conflicts - make sure YOUR jars are loaded first, ahead of
Spark’s jars
• Client mode: use spark-submit options
• —driver-class-path, —driver-library-path
• Spark Classloader order of resolution
• Classes specified via —jars, —packages first (if above flag is set)
• Everything else in SPARK_CLASSPATH
20
Some useful config options
21
spark.serializer org.apache.spark.serializer.KryoSerializer
spark.default.parallelism or pass # partitions for shuffle/reduce tasks as second arg
spark.scheduler.mode
FAIR - enable parallelism within apps (multi-tenant or
low-latency apps like SQL server)
spark.shuffle.memoryFraction,
spark.storage.memoryFraction
Fraction of Java heap to allocate for shuffle and RDD
caching, respectively, before spilling to disk
spark.cleaner.ttl
Enables periodic cleanup of cached RDDs, good for long-
lived jobs
spark.akka.frameSize
Increase the default of 10 (MB) to send back very large
results to the driver app (code smell)
spark.task.maxFailures # of retries for task failure is this - 1
Control Spark SQL Shuffles
• By default, Spark SQL / DataFrames will use 200
partitions when doing any groupBy / distinct operations
• sqlContext.setConf(

"spark.sql.shuffle.partitions", "16")
22
Prevent temp files from filling disks
• (Spark Standalone mode only)
• spark.worker.cleanup.enabled = true
• spark.worker.cleanup.interval
• Configuring executor log file retention/rotation
spark.executor.logs.rolling.maxRetainedFiles = 90
spark.executor.logs.rolling.strategy = time
23
Tuning Spark GC
• Lots of cached RDDs = huge old gen GC cycles, stop-the-world GC
• Know which operations consume more memory (sorts, shuffles)
• Try the new G1GC … avoids whole-heap scans
• -XX:+UseG1GC
• https://databricks.com/blog/2015/05/28/tuning-java-garbage-collection-for-
spark-applications.html
24
Running Spark Applications
25
Run your apps in the cluster
• spark-submit: —deploy-mode cluster
• Spark Job Server: deploy SJS to the cluster
• Drivers and executors are very chatty - want to reduce latency and decrease
chance of networking timeouts
• Want to avoid running jobs on your local machine
26
Automatic Driver Restarts
• Standalone: —deploy-mode cluster —supervise
• YARN: —deploy-mode cluster
• Mesos: use Marathon to restart dead slaves
• Periodic checkpointing: important for recovering data
• RDD checkpointing helps reduce long RDD lineages
27
Speeding up application startup
• Spark-submit’s —packages option is super convenient for downloading
dependencies, but avoid it in production
• Downloads tons of jars from Maven when driver starts up, then executors
copy all the jars from driver
• Deploy frequently used dependencies to worker nodes yourself
• For really fast Spark jobs, use the Spark Job Server and share a SparkContext
amongst jobs!
28
Spark(Context) Metrics
• Spark’s built in MetricsSystem has sources (Spark info, JVM, etc.) and sinks
(Graphite, etc.)
• Configure metrics.properties (template in spark conf/ dir) and use these
params to spark-submit
--files=/path/to/metrics.properties 
--conf spark.metrics.conf=metrics.properties
• See http://www.hammerlab.org/2015/02/27/monitoring-spark-with-graphite-
and-grafana/
29
Application Metrics
• Missing Hadoop counters? Use Spark Accumulators
• https://gist.github.com/ibuenros/
9b94736c2bad2f4b8e23
• Above registers accumulators as a source to Spark’s
MetricsSystem
30
Watch how RDDs are cached
• RDDs cached to disk could slow down computation
31
Are your jobs stuck?
• First check cluster resources - does a job have enough CPU/mem?
• Take a thread dump of executors:
32
TheWorst Killer - Classpath
• Classpath / jar versioning issues may cause Spark to hang silently. Debug
using the Environment tab of the UI:
33
Spark Job Server
34
35
http://github.com/spark-jobserver/spark-jobserver
Open Source!!
Also find it on spark-packages.org
Spark Job Server -What
• REST Interface for your Spark jobs
• Streaming, SQL, extendable
• Job history and configuration logged to a database
• Enable interactive low-latency queries (SQL/Dataframes
works too) of cached RDDs and tables
36
Spark Job Server -Where
37
Kafka
Spark
Streaming
Datastore Spark
Spark Job
Server
Internal users Internet
HTTP/HTTPS
Spark Job Server -Why
• Spark as a service
• Share Spark across the Enterprise
• HTTPS and LDAP Authentication
• Enterprises - easy integration with other teams, any language
• Share in-memory RDDs across logical jobs
• Low-latency queries
38
Used in Production -Worldwide
39
Used in Production
• As of last month, officially included in 

Datastax Enterprise 4.8!
40
Active Community
• Large number of contributions from community
• HTTPS/LDAP contributed by team at KNIME
• Multiple committers
• Gitter IM channel, active Google group
41
Platform Independent
• Spark Standalone
• Mesos
• Yarn
• Docker
• Example: platform-independent LDAP auth, HTTPS, can be
used as a portal
42
ExampleWorkflow
43
Creating a Job Server Project
• sbt assembly -> fat jar -> upload to job server
• "provided" is used. Don’t want SBT assembly to include
the whole job server jar.
• Java projects should be possible too
44
resolvers += "Job Server Bintray" at "https://dl.bintray.com/spark-
jobserver/maven"
libraryDependencies += "spark.jobserver" % "job-server-api" %
"0.5.0" % "provided"
• In your build.sbt, add this
/**
* A super-simple Spark job example that implements the SparkJob trait and
* can be submitted to the job server.
*/
object WordCountExample extends SparkJob {
override def validate(sc: SparkContext, config: Config): SparkJobValidation =
{
Try(config.getString(“input.string”))
.map(x => SparkJobValid)
.getOrElse(SparkJobInvalid(“No input.string”))
}
override def runJob(sc: SparkContext, config: Config): Any = {
val dd = sc.parallelize(config.getString(“input.string”).split(" ").toSeq)
dd.map((_, 1)).reduceByKey(_ + _).collect().toMap
}
}
Example Job Server Job
45
What’s Different?
• Job does not create Context, Job Server does
• Decide when I run the job: in own context, or in pre-created context
• Allows for very modular Spark development
• Break up a giant Spark app into multiple logical jobs
• Example:
• One job to load DataFrames tables
• One job to query them
• One job to run diagnostics and report debugging information
46
Submitting and Running a Job
47
✦ curl --data-binary @../target/mydemo.jar localhost:8090/
jars/demo
OK[11:32 PM] ~
✦ curl -d "input.string = A lazy dog jumped mean dog"
'localhost:8090/jobs?
appName=demo&classPath=WordCountExample&sync=true'
{
"status": "OK",
"RESULT": {
"lazy": 1,
"jumped": 1,
"A": 1,
"mean": 1,
"dog": 2
}
}
Retrieve Job Statuses
48
~/s/jobserver (evan-working-1 ↩=) curl 'localhost:8090/jobs?
limit=2'
[{
"duration": "77.744 secs",
"classPath": "ooyala.cnd.CreateMaterializedView",
"startTime": "2013-11-26T20:13:09.071Z",
"context": "8b7059dd-ooyala.cnd.CreateMaterializedView",
"status": "FINISHED",
"jobId": "9982f961-aaaa-4195-88c2-962eae9b08d9"
}, {
"duration": "58.067 secs",
"classPath": "ooyala.cnd.CreateMaterializedView",
"startTime": "2013-11-26T20:22:03.257Z",
"context": "d0a5ebdc-ooyala.cnd.CreateMaterializedView",
"status": "FINISHED",
"jobId": "e9317383-6a67-41c4-8291-9c140b6d8459"
}]
Use Case: Fast Query Jobs
49
Spark as a Query Engine
• Goal: spark jobs that run in under a second and answers queries on shared
RDD data
• Query params passed in as job config
• Need to minimize context creation overhead
–Thus many jobs sharing the same SparkContext
• On-heap RDD caching means no serialization loss
• Need to consider concurrent jobs (fair scheduling)
50
51
RDDLoad Data Query JobSpark

Executors
Cassandra
REST Job Server
Query Job
Query
Result
Query
Result
new SparkContext
Create
query
context
Load
some
data
Sharing Data Between Jobs
• RDD Caching
–Benefit: no need to serialize data. Especially useful for indexes etc.
–Job server provides a NamedRdds trait for thread-safe CRUD of cached
RDDs by name
• (Compare to SparkContext’s API which uses an integer ID and is not
thread safe)
• For example, at Ooyala a number of fields are multiplexed into the RDD
name: timestamp:customerID:granularity
52
Data Concurrency
• With fair scheduler, multiple Job Server jobs can run simultaneously on one
SparkContext
• Managing multiple updates to RDDs
–Cache keeps track of which RDDs being updated
–Example: thread A spark job creates RDD “A” at t0
–thread B fetches RDD “A” at t1 > t0
–Both threads A and B, using NamedRdds, will get the RDD at time t2 when
thread A finishes creating the RDD “A”
53
Spark SQL/Hive Query Server
✤ Start a context based on SQLContext:

curl -d "" '127.0.0.1:8090/contexts/sql-context?context-
factory=spark.jobserver.context.SQLContextFactory'
✤ Run a job for loading and caching tables in DataFrames

curl -d "" '127.0.0.1:8090/jobs?
appName=test&classPath=spark.jobserver.SqlLoaderJob&context=sql-
context&sync=true'
✤ Supply a query to a Query Job. All queries are logged in database by Spark
Job Server.

curl -d ‘sql=“SELECT count(*) FROM footable”’ '127.0.0.1:8090/jobs?
appName=test&classPath=spark.jobserver.SqlQueryJob&context=sql-
context&sync=true'
54
Example: Combining Streaming And Spark SQL
55
SparkSQLStreamingContext
Kafka Streaming
Job
SQL
Query Job
DataFrames
Spark Job Server
SQL Query
SparkSQLStreamingJob
56
trait SparkSqlStreamingJob extends SparkJobBase {
type C = SQLStreamingContext
}
class SQLStreamingContext(c: SparkContext) {
val streamingContext = new StreamingContext(c, ...)
val sqlContext = new SQLContext(c)
}
Now you have access to both StreamingContext and
SQLContext, and it can be shared across jobs!
SparkSQLStreamingContext
57
To start this context:

curl -d "" “localhost:8090/contexts/stream_sqltest?context-
factory=com.abc.SQLStreamingContextFactory"
class SQLStreamingContextFactory extends SparkContextFactory {
import SparkJobUtils._
type C = SQLStreamingContext with ContextLike
def makeContext(config: Config, contextConfig: Config, contextName: String): C = {
val batchInterval = contextConfig.getInt("batch_interval")
val conf = configToSparkConf(config, contextConfig, contextName)
new SQLStreamingContext(new SparkContext(conf), Seconds(batchInterval)) with ContextLike {
def sparkContext: SparkContext = this.streamingContext.sparkContext
def isValidJob(job: SparkJobBase): Boolean = job.isInstanceOf[SparkSqlStreamingJob]
// Stop the streaming context, but not the SparkContext so that it can be re-used
// to create another streaming context if required:
def stop() { this.streamingContext.stop(false) }
}
}
}
FutureWork
58
Future Plans
• PR: Forked JVMs for supporting many concurrent
contexts
• True HA operation
• Swagger API documentation
59
HA for Job Server
Job
Server 1
Job
Server 2
Active
Job
Context
Gossip
Load balancer
60
Database
GET /jobs/<id>
HA and Hot Failover for Jobs
Job
Server 1
Job
Server 2
Active
Job
Context
HDFS
Standby
Job
Context
Gossip
Checkpoint
61
Thanks for your contributions!
• All of these were community contributed:
–HTTPS and Auth
–saving and retrieving job configuration
–forked JVM per context
• Your contributions are very welcome on Github!
62
63
And Everybody is Hiring!!
Thank you!
WhyWe Needed a Job Server
• Our vision for Spark is as a multi-team big data service
• What gets repeated by every team:
• Bastion box for running Hadoop/Spark jobs
• Deploys and process monitoring
• Tracking and serializing job status, progress, and job results
• Job validation
• No easy way to kill jobs
• Polyglot technology stack - Ruby scripts run jobs, Go services
Architecture
Completely Async Design
✤ http://spray.io - probably the fastest JVM HTTP
microframework
✤ Akka Actor based, non blocking
✤ Futures used to manage individual jobs. (Note that
Spark is using Scala futures to manage job stages now)
✤ Single JVM for now, but easy to distribute later via
remote Actors / Akka Cluster
Async Actor Flow
Spray web
API
Request
actor
Local
Supervisor
Job
Manager
Job 1
Future
Job 2
Future
Job Status
Actor
Job Result
Actor
Message flow fully documented
Production Usage
Metadata Store
✤ JarInfo, JobInfo, ConfigInfo
✤ JobSqlDAO. Store metadata to SQL database by JDBC interface.
✤ Easily configured by spark.sqldao.jdbc.url
✤ jdbc:mysql://dbserver:3306/jobserverdb
✤ Multiple Job Servers can share the same MySQL.
✤ Jars uploaded once but accessible by all servers.
✤ The default will be JobSqlDAO and H2.
✤ Single H2 DB file. Serialization and deserialization are handled by H2.
Deployment and Metrics
✤ spark-jobserver repo comes with a full suite of tests and
deploy scripts:
✤ server_deploy.sh for regular server pushes
✤ server_package.sh for Mesos and Chronos .tar.gz
✤ /metricz route for codahale-metrics monitoring
✤ /healthz route for health check
Challenges and Lessons
• Spark is based around contexts - we need a Job Server oriented around logical
jobs
• Running multiple SparkContexts in the same process
• Better long term solution is forked JVM per SparkContext
• Workaround: spark.driver.allowMultipleContexts = true
• Dynamic jar and class loading is tricky
• Manage threads carefully - each context uses lots of threads

Weitere ähnliche Inhalte

Was ist angesagt?

Was ist angesagt? (20)

Processing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeekProcessing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeek
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 
Building a SIMD Supported Vectorized Native Engine for Spark SQL
Building a SIMD Supported Vectorized Native Engine for Spark SQLBuilding a SIMD Supported Vectorized Native Engine for Spark SQL
Building a SIMD Supported Vectorized Native Engine for Spark SQL
 
Why your Spark job is failing
Why your Spark job is failingWhy your Spark job is failing
Why your Spark job is failing
 
Making Nested Columns as First Citizen in Apache Spark SQL
Making Nested Columns as First Citizen in Apache Spark SQLMaking Nested Columns as First Citizen in Apache Spark SQL
Making Nested Columns as First Citizen in Apache Spark SQL
 
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital KediaTuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache Spark
 
Spark Summit EU 2015: Lessons from 300+ production users
Spark Summit EU 2015: Lessons from 300+ production usersSpark Summit EU 2015: Lessons from 300+ production users
Spark Summit EU 2015: Lessons from 300+ production users
 
Debugging PySpark: Spark Summit East talk by Holden Karau
Debugging PySpark: Spark Summit East talk by Holden KarauDebugging PySpark: Spark Summit East talk by Holden Karau
Debugging PySpark: Spark Summit East talk by Holden Karau
 
Physical Plans in Spark SQL
Physical Plans in Spark SQLPhysical Plans in Spark SQL
Physical Plans in Spark SQL
 
Understanding and Improving Code Generation
Understanding and Improving Code GenerationUnderstanding and Improving Code Generation
Understanding and Improving Code Generation
 
Spark Summit EU talk by Ted Malaska
Spark Summit EU talk by Ted MalaskaSpark Summit EU talk by Ted Malaska
Spark Summit EU talk by Ted Malaska
 
Monitor Apache Spark 3 on Kubernetes using Metrics and Plugins
Monitor Apache Spark 3 on Kubernetes using Metrics and PluginsMonitor Apache Spark 3 on Kubernetes using Metrics and Plugins
Monitor Apache Spark 3 on Kubernetes using Metrics and Plugins
 
What is Apache Spark | Apache Spark Tutorial For Beginners | Apache Spark Tra...
What is Apache Spark | Apache Spark Tutorial For Beginners | Apache Spark Tra...What is Apache Spark | Apache Spark Tutorial For Beginners | Apache Spark Tra...
What is Apache Spark | Apache Spark Tutorial For Beginners | Apache Spark Tra...
 
Introduction to apache spark
Introduction to apache spark Introduction to apache spark
Introduction to apache spark
 
Beyond SQL: Speeding up Spark with DataFrames
Beyond SQL: Speeding up Spark with DataFramesBeyond SQL: Speeding up Spark with DataFrames
Beyond SQL: Speeding up Spark with DataFrames
 
Introduction to Apache Spark Developer Training
Introduction to Apache Spark Developer TrainingIntroduction to Apache Spark Developer Training
Introduction to Apache Spark Developer Training
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introduction
 
Introduction to Spark (Intern Event Presentation)
Introduction to Spark (Intern Event Presentation)Introduction to Spark (Intern Event Presentation)
Introduction to Spark (Intern Event Presentation)
 
Working with JSON Data in PostgreSQL vs. MongoDB
Working with JSON Data in PostgreSQL vs. MongoDBWorking with JSON Data in PostgreSQL vs. MongoDB
Working with JSON Data in PostgreSQL vs. MongoDB
 

Andere mochten auch

Spark Summit Europe: Building a REST Job Server for interactive Spark as a se...
Spark Summit Europe: Building a REST Job Server for interactive Spark as a se...Spark Summit Europe: Building a REST Job Server for interactive Spark as a se...
Spark Summit Europe: Building a REST Job Server for interactive Spark as a se...
gethue
 
Dynamically Allocate Cluster Resources to your Spark Application
Dynamically Allocate Cluster Resources to your Spark ApplicationDynamically Allocate Cluster Resources to your Spark Application
Dynamically Allocate Cluster Resources to your Spark Application
DataWorks Summit
 
Spark-on-YARN: Empower Spark Applications on Hadoop Cluster
Spark-on-YARN: Empower Spark Applications on Hadoop ClusterSpark-on-YARN: Empower Spark Applications on Hadoop Cluster
Spark-on-YARN: Empower Spark Applications on Hadoop Cluster
DataWorks Summit
 

Andere mochten auch (14)

Get most out of Spark on YARN
Get most out of Spark on YARNGet most out of Spark on YARN
Get most out of Spark on YARN
 
Spark Compute as a Service at Paypal with Prabhu Kasinathan
Spark Compute as a Service at Paypal with Prabhu KasinathanSpark Compute as a Service at Paypal with Prabhu Kasinathan
Spark Compute as a Service at Paypal with Prabhu Kasinathan
 
Hadoop and Spark Analytics over Better Storage
Hadoop and Spark Analytics over Better StorageHadoop and Spark Analytics over Better Storage
Hadoop and Spark Analytics over Better Storage
 
Producing Spark on YARN for ETL
Producing Spark on YARN for ETLProducing Spark on YARN for ETL
Producing Spark on YARN for ETL
 
SocSciBot(01 Mar2010) - Korean Manual
SocSciBot(01 Mar2010) - Korean ManualSocSciBot(01 Mar2010) - Korean Manual
SocSciBot(01 Mar2010) - Korean Manual
 
Spark Summit Europe: Building a REST Job Server for interactive Spark as a se...
Spark Summit Europe: Building a REST Job Server for interactive Spark as a se...Spark Summit Europe: Building a REST Job Server for interactive Spark as a se...
Spark Summit Europe: Building a REST Job Server for interactive Spark as a se...
 
Spark on yarn
Spark on yarnSpark on yarn
Spark on yarn
 
Dynamically Allocate Cluster Resources to your Spark Application
Dynamically Allocate Cluster Resources to your Spark ApplicationDynamically Allocate Cluster Resources to your Spark Application
Dynamically Allocate Cluster Resources to your Spark Application
 
Spark-on-YARN: Empower Spark Applications on Hadoop Cluster
Spark-on-YARN: Empower Spark Applications on Hadoop ClusterSpark-on-YARN: Empower Spark Applications on Hadoop Cluster
Spark-on-YARN: Empower Spark Applications on Hadoop Cluster
 
ETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetup
 
Proxy Servers
Proxy ServersProxy Servers
Proxy Servers
 
Apache Spark Model Deployment
Apache Spark Model Deployment Apache Spark Model Deployment
Apache Spark Model Deployment
 
Proxy Server
Proxy ServerProxy Server
Proxy Server
 
Spark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting GuideSpark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting Guide
 

Ähnlich wie Productionizing Spark and the Spark Job Server

Ähnlich wie Productionizing Spark and the Spark Job Server (20)

Productionizing Spark and the REST Job Server- Evan Chan
Productionizing Spark and the REST Job Server- Evan ChanProductionizing Spark and the REST Job Server- Evan Chan
Productionizing Spark and the REST Job Server- Evan Chan
 
Introduction to Apache Spark :: Lagos Scala Meetup session 2
Introduction to Apache Spark :: Lagos Scala Meetup session 2 Introduction to Apache Spark :: Lagos Scala Meetup session 2
Introduction to Apache Spark :: Lagos Scala Meetup session 2
 
Apache Spark Fundamentals
Apache Spark FundamentalsApache Spark Fundamentals
Apache Spark Fundamentals
 
Intro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of TwingoIntro to Apache Spark by CTO of Twingo
Intro to Apache Spark by CTO of Twingo
 
Real time Analytics with Apache Kafka and Apache Spark
Real time Analytics with Apache Kafka and Apache SparkReal time Analytics with Apache Kafka and Apache Spark
Real time Analytics with Apache Kafka and Apache Spark
 
Lessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and MicroservicesLessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and Microservices
 
Tuning and Monitoring Deep Learning on Apache Spark
Tuning and Monitoring Deep Learning on Apache SparkTuning and Monitoring Deep Learning on Apache Spark
Tuning and Monitoring Deep Learning on Apache Spark
 
Apache Spark Core
Apache Spark CoreApache Spark Core
Apache Spark Core
 
Programming in Spark using PySpark
Programming in Spark using PySpark      Programming in Spark using PySpark
Programming in Spark using PySpark
 
Apache Spark Tutorial
Apache Spark TutorialApache Spark Tutorial
Apache Spark Tutorial
 
East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning
 East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning
East Bay Java User Group Oct 2014 Spark Streaming Kinesis Machine Learning
 
Lessons Learned From Running Spark On Docker
Lessons Learned From Running Spark On DockerLessons Learned From Running Spark On Docker
Lessons Learned From Running Spark On Docker
 
Spark Summit EU talk by Luca Canali
Spark Summit EU talk by Luca CanaliSpark Summit EU talk by Luca Canali
Spark Summit EU talk by Luca Canali
 
How to deploy Apache Spark 
to Mesos/DCOS
How to deploy Apache Spark 
to Mesos/DCOSHow to deploy Apache Spark 
to Mesos/DCOS
How to deploy Apache Spark 
to Mesos/DCOS
 
Intro to Apache Spark
Intro to Apache SparkIntro to Apache Spark
Intro to Apache Spark
 
Intro to Apache Spark
Intro to Apache SparkIntro to Apache Spark
Intro to Apache Spark
 
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
Global Big Data Conference Sept 2014 AWS Kinesis Spark Streaming Approximatio...
 
Spark as a Platform to Support Multi-Tenancy and Many Kinds of Data Applicati...
Spark as a Platform to Support Multi-Tenancy and Many Kinds of Data Applicati...Spark as a Platform to Support Multi-Tenancy and Many Kinds of Data Applicati...
Spark as a Platform to Support Multi-Tenancy and Many Kinds of Data Applicati...
 
Apache Spark in Industry
Apache Spark in IndustryApache Spark in Industry
Apache Spark in Industry
 
Migrating ETL Workflow to Apache Spark at Scale in Pinterest
Migrating ETL Workflow to Apache Spark at Scale in PinterestMigrating ETL Workflow to Apache Spark at Scale in Pinterest
Migrating ETL Workflow to Apache Spark at Scale in Pinterest
 

Mehr von Evan Chan

Spark Summit 2014: Spark Job Server Talk
Spark Summit 2014:  Spark Job Server TalkSpark Summit 2014:  Spark Job Server Talk
Spark Summit 2014: Spark Job Server Talk
Evan Chan
 

Mehr von Evan Chan (16)

Porting a Streaming Pipeline from Scala to Rust
Porting a Streaming Pipeline from Scala to RustPorting a Streaming Pipeline from Scala to Rust
Porting a Streaming Pipeline from Scala to Rust
 
Designing Stateful Apps for Cloud and Kubernetes
Designing Stateful Apps for Cloud and KubernetesDesigning Stateful Apps for Cloud and Kubernetes
Designing Stateful Apps for Cloud and Kubernetes
 
Histograms at scale - Monitorama 2019
Histograms at scale - Monitorama 2019Histograms at scale - Monitorama 2019
Histograms at scale - Monitorama 2019
 
FiloDB: Reactive, Real-Time, In-Memory Time Series at Scale
FiloDB: Reactive, Real-Time, In-Memory Time Series at ScaleFiloDB: Reactive, Real-Time, In-Memory Time Series at Scale
FiloDB: Reactive, Real-Time, In-Memory Time Series at Scale
 
Building a High-Performance Database with Scala, Akka, and Spark
Building a High-Performance Database with Scala, Akka, and SparkBuilding a High-Performance Database with Scala, Akka, and Spark
Building a High-Performance Database with Scala, Akka, and Spark
 
700 Updatable Queries Per Second: Spark as a Real-Time Web Service
700 Updatable Queries Per Second: Spark as a Real-Time Web Service700 Updatable Queries Per Second: Spark as a Real-Time Web Service
700 Updatable Queries Per Second: Spark as a Real-Time Web Service
 
Building Scalable Data Pipelines - 2016 DataPalooza Seattle
Building Scalable Data Pipelines - 2016 DataPalooza SeattleBuilding Scalable Data Pipelines - 2016 DataPalooza Seattle
Building Scalable Data Pipelines - 2016 DataPalooza Seattle
 
FiloDB - Breakthrough OLAP Performance with Cassandra and Spark
FiloDB - Breakthrough OLAP Performance with Cassandra and SparkFiloDB - Breakthrough OLAP Performance with Cassandra and Spark
FiloDB - Breakthrough OLAP Performance with Cassandra and Spark
 
Breakthrough OLAP performance with Cassandra and Spark
Breakthrough OLAP performance with Cassandra and SparkBreakthrough OLAP performance with Cassandra and Spark
Breakthrough OLAP performance with Cassandra and Spark
 
Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015
 
MIT lecture - Socrata Open Data Architecture
MIT lecture - Socrata Open Data ArchitectureMIT lecture - Socrata Open Data Architecture
MIT lecture - Socrata Open Data Architecture
 
OLAP with Cassandra and Spark
OLAP with Cassandra and SparkOLAP with Cassandra and Spark
OLAP with Cassandra and Spark
 
Spark Summit 2014: Spark Job Server Talk
Spark Summit 2014:  Spark Job Server TalkSpark Summit 2014:  Spark Job Server Talk
Spark Summit 2014: Spark Job Server Talk
 
Spark Job Server and Spark as a Query Engine (Spark Meetup 5/14)
Spark Job Server and Spark as a Query Engine (Spark Meetup 5/14)Spark Job Server and Spark as a Query Engine (Spark Meetup 5/14)
Spark Job Server and Spark as a Query Engine (Spark Meetup 5/14)
 
Cassandra Day 2014: Interactive Analytics with Cassandra and Spark
Cassandra Day 2014: Interactive Analytics with Cassandra and SparkCassandra Day 2014: Interactive Analytics with Cassandra and Spark
Cassandra Day 2014: Interactive Analytics with Cassandra and Spark
 
Real-time Analytics with Cassandra, Spark, and Shark
Real-time Analytics with Cassandra, Spark, and SharkReal-time Analytics with Cassandra, Spark, and Shark
Real-time Analytics with Cassandra, Spark, and Shark
 

Kürzlich hochgeladen

Cara Menggugurkan Sperma Yang Masuk Rahim Biyar Tidak Hamil
Cara Menggugurkan Sperma Yang Masuk Rahim Biyar Tidak HamilCara Menggugurkan Sperma Yang Masuk Rahim Biyar Tidak Hamil
Cara Menggugurkan Sperma Yang Masuk Rahim Biyar Tidak Hamil
Cara Menggugurkan Kandungan 087776558899
 
"Lesotho Leaps Forward: A Chronicle of Transformative Developments"
"Lesotho Leaps Forward: A Chronicle of Transformative Developments""Lesotho Leaps Forward: A Chronicle of Transformative Developments"
"Lesotho Leaps Forward: A Chronicle of Transformative Developments"
mphochane1998
 
1_Introduction + EAM Vocabulary + how to navigate in EAM.pdf
1_Introduction + EAM Vocabulary + how to navigate in EAM.pdf1_Introduction + EAM Vocabulary + how to navigate in EAM.pdf
1_Introduction + EAM Vocabulary + how to navigate in EAM.pdf
AldoGarca30
 
DeepFakes presentation : brief idea of DeepFakes
DeepFakes presentation : brief idea of DeepFakesDeepFakes presentation : brief idea of DeepFakes
DeepFakes presentation : brief idea of DeepFakes
MayuraD1
 
Integrated Test Rig For HTFE-25 - Neometrix
Integrated Test Rig For HTFE-25 - NeometrixIntegrated Test Rig For HTFE-25 - Neometrix
Integrated Test Rig For HTFE-25 - Neometrix
Neometrix_Engineering_Pvt_Ltd
 
Verification of thevenin's theorem for BEEE Lab (1).pptx
Verification of thevenin's theorem for BEEE Lab (1).pptxVerification of thevenin's theorem for BEEE Lab (1).pptx
Verification of thevenin's theorem for BEEE Lab (1).pptx
chumtiyababu
 
Kuwait City MTP kit ((+919101817206)) Buy Abortion Pills Kuwait
Kuwait City MTP kit ((+919101817206)) Buy Abortion Pills KuwaitKuwait City MTP kit ((+919101817206)) Buy Abortion Pills Kuwait
Kuwait City MTP kit ((+919101817206)) Buy Abortion Pills Kuwait
jaanualu31
 
Call Girls in South Ex (delhi) call me [🔝9953056974🔝] escort service 24X7
Call Girls in South Ex (delhi) call me [🔝9953056974🔝] escort service 24X7Call Girls in South Ex (delhi) call me [🔝9953056974🔝] escort service 24X7
Call Girls in South Ex (delhi) call me [🔝9953056974🔝] escort service 24X7
9953056974 Low Rate Call Girls In Saket, Delhi NCR
 
Hospital management system project report.pdf
Hospital management system project report.pdfHospital management system project report.pdf
Hospital management system project report.pdf
Kamal Acharya
 

Kürzlich hochgeladen (20)

Navigating Complexity: The Role of Trusted Partners and VIAS3D in Dassault Sy...
Navigating Complexity: The Role of Trusted Partners and VIAS3D in Dassault Sy...Navigating Complexity: The Role of Trusted Partners and VIAS3D in Dassault Sy...
Navigating Complexity: The Role of Trusted Partners and VIAS3D in Dassault Sy...
 
Cara Menggugurkan Sperma Yang Masuk Rahim Biyar Tidak Hamil
Cara Menggugurkan Sperma Yang Masuk Rahim Biyar Tidak HamilCara Menggugurkan Sperma Yang Masuk Rahim Biyar Tidak Hamil
Cara Menggugurkan Sperma Yang Masuk Rahim Biyar Tidak Hamil
 
Work-Permit-Receiver-in-Saudi-Aramco.pptx
Work-Permit-Receiver-in-Saudi-Aramco.pptxWork-Permit-Receiver-in-Saudi-Aramco.pptx
Work-Permit-Receiver-in-Saudi-Aramco.pptx
 
HAND TOOLS USED AT ELECTRONICS WORK PRESENTED BY KOUSTAV SARKAR
HAND TOOLS USED AT ELECTRONICS WORK PRESENTED BY KOUSTAV SARKARHAND TOOLS USED AT ELECTRONICS WORK PRESENTED BY KOUSTAV SARKAR
HAND TOOLS USED AT ELECTRONICS WORK PRESENTED BY KOUSTAV SARKAR
 
A CASE STUDY ON CERAMIC INDUSTRY OF BANGLADESH.pptx
A CASE STUDY ON CERAMIC INDUSTRY OF BANGLADESH.pptxA CASE STUDY ON CERAMIC INDUSTRY OF BANGLADESH.pptx
A CASE STUDY ON CERAMIC INDUSTRY OF BANGLADESH.pptx
 
HOA1&2 - Module 3 - PREHISTORCI ARCHITECTURE OF KERALA.pptx
HOA1&2 - Module 3 - PREHISTORCI ARCHITECTURE OF KERALA.pptxHOA1&2 - Module 3 - PREHISTORCI ARCHITECTURE OF KERALA.pptx
HOA1&2 - Module 3 - PREHISTORCI ARCHITECTURE OF KERALA.pptx
 
"Lesotho Leaps Forward: A Chronicle of Transformative Developments"
"Lesotho Leaps Forward: A Chronicle of Transformative Developments""Lesotho Leaps Forward: A Chronicle of Transformative Developments"
"Lesotho Leaps Forward: A Chronicle of Transformative Developments"
 
S1S2 B.Arch MGU - HOA1&2 Module 3 -Temple Architecture of Kerala.pptx
S1S2 B.Arch MGU - HOA1&2 Module 3 -Temple Architecture of Kerala.pptxS1S2 B.Arch MGU - HOA1&2 Module 3 -Temple Architecture of Kerala.pptx
S1S2 B.Arch MGU - HOA1&2 Module 3 -Temple Architecture of Kerala.pptx
 
1_Introduction + EAM Vocabulary + how to navigate in EAM.pdf
1_Introduction + EAM Vocabulary + how to navigate in EAM.pdf1_Introduction + EAM Vocabulary + how to navigate in EAM.pdf
1_Introduction + EAM Vocabulary + how to navigate in EAM.pdf
 
Hostel management system project report..pdf
Hostel management system project report..pdfHostel management system project report..pdf
Hostel management system project report..pdf
 
Employee leave management system project.
Employee leave management system project.Employee leave management system project.
Employee leave management system project.
 
DeepFakes presentation : brief idea of DeepFakes
DeepFakes presentation : brief idea of DeepFakesDeepFakes presentation : brief idea of DeepFakes
DeepFakes presentation : brief idea of DeepFakes
 
Integrated Test Rig For HTFE-25 - Neometrix
Integrated Test Rig For HTFE-25 - NeometrixIntegrated Test Rig For HTFE-25 - Neometrix
Integrated Test Rig For HTFE-25 - Neometrix
 
Verification of thevenin's theorem for BEEE Lab (1).pptx
Verification of thevenin's theorem for BEEE Lab (1).pptxVerification of thevenin's theorem for BEEE Lab (1).pptx
Verification of thevenin's theorem for BEEE Lab (1).pptx
 
Kuwait City MTP kit ((+919101817206)) Buy Abortion Pills Kuwait
Kuwait City MTP kit ((+919101817206)) Buy Abortion Pills KuwaitKuwait City MTP kit ((+919101817206)) Buy Abortion Pills Kuwait
Kuwait City MTP kit ((+919101817206)) Buy Abortion Pills Kuwait
 
Design For Accessibility: Getting it right from the start
Design For Accessibility: Getting it right from the startDesign For Accessibility: Getting it right from the start
Design For Accessibility: Getting it right from the start
 
Tamil Call Girls Bhayandar WhatsApp +91-9930687706, Best Service
Tamil Call Girls Bhayandar WhatsApp +91-9930687706, Best ServiceTamil Call Girls Bhayandar WhatsApp +91-9930687706, Best Service
Tamil Call Girls Bhayandar WhatsApp +91-9930687706, Best Service
 
Call Girls in South Ex (delhi) call me [🔝9953056974🔝] escort service 24X7
Call Girls in South Ex (delhi) call me [🔝9953056974🔝] escort service 24X7Call Girls in South Ex (delhi) call me [🔝9953056974🔝] escort service 24X7
Call Girls in South Ex (delhi) call me [🔝9953056974🔝] escort service 24X7
 
Unleashing the Power of the SORA AI lastest leap
Unleashing the Power of the SORA AI lastest leapUnleashing the Power of the SORA AI lastest leap
Unleashing the Power of the SORA AI lastest leap
 
Hospital management system project report.pdf
Hospital management system project report.pdfHospital management system project report.pdf
Hospital management system project report.pdf
 

Productionizing Spark and the Spark Job Server

  • 1. Productionizing Spark
 and the Spark REST Job Server Evan Chan Distinguished Engineer @TupleJump
  • 2. Who am I • Distinguished Engineer, Tuplejump • @evanfchan • http://github.com/velvia • User and contributor to Spark since 0.9 • Co-creator and maintainer of Spark Job Server 2
  • 3. TupleJump ✤ Tuplejump is a big data technology leader providing solutions and development partnership. ✤ FiloDB - Spark-based analytics database for time series and event data (github.com/tuplejump/FiloDB) ✤ Calliope - the first Spark-Cassandra integration ✤ Stargate - an open source Lucene indexer for Cassandra ✤ SnackFS - open source HDFS for Cassandra 3
  • 4. TupleJump - Big Data Dev Partners 4
  • 6. Choices, choices, choices • YARN, Mesos, Standalone? • With a distribution? • What environment? • How should I deploy? • Hosted options? • What about dependencies? 6
  • 7. BasicTerminology • The Spark documentation is really quite good. 7
  • 8. What all the clusters have in common • YARN, Mesos, and Standalone all support the following features: –Running the Spark driver app in cluster mode –Restarts of the driver app upon failure –UI to examine state of workers and apps 8
  • 9. Spark Standalone Mode • The easiest clustering mode to deploy** –Use make-distribution.sh to package, copy to all nodes –sbin/start-master.sh on master node, then start slaves –Test with spark-shell • HA Master through Zookeeper election • Must dedicate whole cluster to Spark • In latest survey, used by almost half of Spark users 9
  • 10. Apache Mesos • Was started by Matias in 2007 before he worked on Spark! • Can run your entire company on Mesos, not just big data –Great support for micro services - Docker, Marathon –Can run non-JVM workloads like MPI • Commercial backing from Mesosphere • Heavily used at Twitter and AirBNB • The Mesosphere DCOS will revolutionize Spark et al deployment - “dcos package install spark” !! 10
  • 11. Mesos vsYARN • Mesos is a two-level resource manager, with pluggable schedulers –You can run YARN on Mesos, with YARN delegating resource offers to Mesos (Project Myriad) –You can run multiple schedulers within Mesos, and write your own • If you’re already a Hadoop / Cloudera etc shop, YARN is easy choice • If you’re starting out, go 100% Mesos 11
  • 12. Mesos Coarse vs Fine-Grained • Spark offers two modes to run Mesos Spark apps in (and you can choose per driver app): –coarse-grained: Spark allocates fixed number of workers for duration of driver app –fine-grained (default): Dynamic executor allocation per task, but higher overhead per task • Use coarse-grained if you run low-latency jobs 12
  • 13. What about Datastax DSE? • Cassandra, Hadoop, Spark all bundled in one distribution, collocated • Custom cluster manager and HA/failover logic for Spark Master, using Cassandra gossip • Can use CFS (Cassandra-based HDFS), SnackFS, or plain Cassandra tables for storage –or use Tachyon to cache, then no need to collocate (use Mesosphere DCOS) 13
  • 14. Hosted Apache Spark • Spark on Amazon EMR - first class citizen now –Direct S3 access! • Google Compute Engine - “Click to Deploy” Hadoop+Spark • Databricks Cloud • Many more coming • What you notice about the different environments: –Everybody has their own way of starting: spark-submit vs dse spark vs aws emr … vs dcos spark … 14
  • 15. Mesosphere DCOS • Automates deployment to AWS, Google, etc. • Common API and UI, better cost and control, cloud • Load balancing and routing, Mesos for resource sharing • dcos package install spark 15
  • 17. Building Spark • Make sure you build for the right Hadoop version • eg mvn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package • Make sure you build for the right Scala version - Spark supports both 2.10 and 2.11 17
  • 18. Jars schmars • Dependency conflicts are the worst part of Spark dev • Every distro has slightly different jars - eg CDH < 5.4 packaged a different version of Akka • Leave out Hive if you don’t need it • Use the Spark UI “Environment” tab to check jars and how they got there • spark-submit —jars / —packages forwards jars to every executor (unless it’s an HDFS / HTTP path) • Spark-env.sh SPARK_CLASSPATH - include dep jars you’ve deployed to every node 18
  • 19. Jars schmars • You don’t need to package every dependency with your Spark application! • spark-streaming is included in the distribution • spark-streaming includes some Kafka jars already • etc. 19
  • 20. ClassPath Configuration Options • spark.driver.userClassPathFirst, spark.executor.userClassPathFirst • One way to solve dependency conflicts - make sure YOUR jars are loaded first, ahead of Spark’s jars • Client mode: use spark-submit options • —driver-class-path, —driver-library-path • Spark Classloader order of resolution • Classes specified via —jars, —packages first (if above flag is set) • Everything else in SPARK_CLASSPATH 20
  • 21. Some useful config options 21 spark.serializer org.apache.spark.serializer.KryoSerializer spark.default.parallelism or pass # partitions for shuffle/reduce tasks as second arg spark.scheduler.mode FAIR - enable parallelism within apps (multi-tenant or low-latency apps like SQL server) spark.shuffle.memoryFraction, spark.storage.memoryFraction Fraction of Java heap to allocate for shuffle and RDD caching, respectively, before spilling to disk spark.cleaner.ttl Enables periodic cleanup of cached RDDs, good for long- lived jobs spark.akka.frameSize Increase the default of 10 (MB) to send back very large results to the driver app (code smell) spark.task.maxFailures # of retries for task failure is this - 1
  • 22. Control Spark SQL Shuffles • By default, Spark SQL / DataFrames will use 200 partitions when doing any groupBy / distinct operations • sqlContext.setConf(
 "spark.sql.shuffle.partitions", "16") 22
  • 23. Prevent temp files from filling disks • (Spark Standalone mode only) • spark.worker.cleanup.enabled = true • spark.worker.cleanup.interval • Configuring executor log file retention/rotation spark.executor.logs.rolling.maxRetainedFiles = 90 spark.executor.logs.rolling.strategy = time 23
  • 24. Tuning Spark GC • Lots of cached RDDs = huge old gen GC cycles, stop-the-world GC • Know which operations consume more memory (sorts, shuffles) • Try the new G1GC … avoids whole-heap scans • -XX:+UseG1GC • https://databricks.com/blog/2015/05/28/tuning-java-garbage-collection-for- spark-applications.html 24
  • 26. Run your apps in the cluster • spark-submit: —deploy-mode cluster • Spark Job Server: deploy SJS to the cluster • Drivers and executors are very chatty - want to reduce latency and decrease chance of networking timeouts • Want to avoid running jobs on your local machine 26
  • 27. Automatic Driver Restarts • Standalone: —deploy-mode cluster —supervise • YARN: —deploy-mode cluster • Mesos: use Marathon to restart dead slaves • Periodic checkpointing: important for recovering data • RDD checkpointing helps reduce long RDD lineages 27
  • 28. Speeding up application startup • Spark-submit’s —packages option is super convenient for downloading dependencies, but avoid it in production • Downloads tons of jars from Maven when driver starts up, then executors copy all the jars from driver • Deploy frequently used dependencies to worker nodes yourself • For really fast Spark jobs, use the Spark Job Server and share a SparkContext amongst jobs! 28
  • 29. Spark(Context) Metrics • Spark’s built in MetricsSystem has sources (Spark info, JVM, etc.) and sinks (Graphite, etc.) • Configure metrics.properties (template in spark conf/ dir) and use these params to spark-submit --files=/path/to/metrics.properties --conf spark.metrics.conf=metrics.properties • See http://www.hammerlab.org/2015/02/27/monitoring-spark-with-graphite- and-grafana/ 29
  • 30. Application Metrics • Missing Hadoop counters? Use Spark Accumulators • https://gist.github.com/ibuenros/ 9b94736c2bad2f4b8e23 • Above registers accumulators as a source to Spark’s MetricsSystem 30
  • 31. Watch how RDDs are cached • RDDs cached to disk could slow down computation 31
  • 32. Are your jobs stuck? • First check cluster resources - does a job have enough CPU/mem? • Take a thread dump of executors: 32
  • 33. TheWorst Killer - Classpath • Classpath / jar versioning issues may cause Spark to hang silently. Debug using the Environment tab of the UI: 33
  • 36. Spark Job Server -What • REST Interface for your Spark jobs • Streaming, SQL, extendable • Job history and configuration logged to a database • Enable interactive low-latency queries (SQL/Dataframes works too) of cached RDDs and tables 36
  • 37. Spark Job Server -Where 37 Kafka Spark Streaming Datastore Spark Spark Job Server Internal users Internet HTTP/HTTPS
  • 38. Spark Job Server -Why • Spark as a service • Share Spark across the Enterprise • HTTPS and LDAP Authentication • Enterprises - easy integration with other teams, any language • Share in-memory RDDs across logical jobs • Low-latency queries 38
  • 39. Used in Production -Worldwide 39
  • 40. Used in Production • As of last month, officially included in 
 Datastax Enterprise 4.8! 40
  • 41. Active Community • Large number of contributions from community • HTTPS/LDAP contributed by team at KNIME • Multiple committers • Gitter IM channel, active Google group 41
  • 42. Platform Independent • Spark Standalone • Mesos • Yarn • Docker • Example: platform-independent LDAP auth, HTTPS, can be used as a portal 42
  • 44. Creating a Job Server Project • sbt assembly -> fat jar -> upload to job server • "provided" is used. Don’t want SBT assembly to include the whole job server jar. • Java projects should be possible too 44 resolvers += "Job Server Bintray" at "https://dl.bintray.com/spark- jobserver/maven" libraryDependencies += "spark.jobserver" % "job-server-api" % "0.5.0" % "provided" • In your build.sbt, add this
  • 45. /** * A super-simple Spark job example that implements the SparkJob trait and * can be submitted to the job server. */ object WordCountExample extends SparkJob { override def validate(sc: SparkContext, config: Config): SparkJobValidation = { Try(config.getString(“input.string”)) .map(x => SparkJobValid) .getOrElse(SparkJobInvalid(“No input.string”)) } override def runJob(sc: SparkContext, config: Config): Any = { val dd = sc.parallelize(config.getString(“input.string”).split(" ").toSeq) dd.map((_, 1)).reduceByKey(_ + _).collect().toMap } } Example Job Server Job 45
  • 46. What’s Different? • Job does not create Context, Job Server does • Decide when I run the job: in own context, or in pre-created context • Allows for very modular Spark development • Break up a giant Spark app into multiple logical jobs • Example: • One job to load DataFrames tables • One job to query them • One job to run diagnostics and report debugging information 46
  • 47. Submitting and Running a Job 47 ✦ curl --data-binary @../target/mydemo.jar localhost:8090/ jars/demo OK[11:32 PM] ~ ✦ curl -d "input.string = A lazy dog jumped mean dog" 'localhost:8090/jobs? appName=demo&classPath=WordCountExample&sync=true' { "status": "OK", "RESULT": { "lazy": 1, "jumped": 1, "A": 1, "mean": 1, "dog": 2 } }
  • 48. Retrieve Job Statuses 48 ~/s/jobserver (evan-working-1 ↩=) curl 'localhost:8090/jobs? limit=2' [{ "duration": "77.744 secs", "classPath": "ooyala.cnd.CreateMaterializedView", "startTime": "2013-11-26T20:13:09.071Z", "context": "8b7059dd-ooyala.cnd.CreateMaterializedView", "status": "FINISHED", "jobId": "9982f961-aaaa-4195-88c2-962eae9b08d9" }, { "duration": "58.067 secs", "classPath": "ooyala.cnd.CreateMaterializedView", "startTime": "2013-11-26T20:22:03.257Z", "context": "d0a5ebdc-ooyala.cnd.CreateMaterializedView", "status": "FINISHED", "jobId": "e9317383-6a67-41c4-8291-9c140b6d8459" }]
  • 49. Use Case: Fast Query Jobs 49
  • 50. Spark as a Query Engine • Goal: spark jobs that run in under a second and answers queries on shared RDD data • Query params passed in as job config • Need to minimize context creation overhead –Thus many jobs sharing the same SparkContext • On-heap RDD caching means no serialization loss • Need to consider concurrent jobs (fair scheduling) 50
  • 51. 51 RDDLoad Data Query JobSpark
 Executors Cassandra REST Job Server Query Job Query Result Query Result new SparkContext Create query context Load some data
  • 52. Sharing Data Between Jobs • RDD Caching –Benefit: no need to serialize data. Especially useful for indexes etc. –Job server provides a NamedRdds trait for thread-safe CRUD of cached RDDs by name • (Compare to SparkContext’s API which uses an integer ID and is not thread safe) • For example, at Ooyala a number of fields are multiplexed into the RDD name: timestamp:customerID:granularity 52
  • 53. Data Concurrency • With fair scheduler, multiple Job Server jobs can run simultaneously on one SparkContext • Managing multiple updates to RDDs –Cache keeps track of which RDDs being updated –Example: thread A spark job creates RDD “A” at t0 –thread B fetches RDD “A” at t1 > t0 –Both threads A and B, using NamedRdds, will get the RDD at time t2 when thread A finishes creating the RDD “A” 53
  • 54. Spark SQL/Hive Query Server ✤ Start a context based on SQLContext:
 curl -d "" '127.0.0.1:8090/contexts/sql-context?context- factory=spark.jobserver.context.SQLContextFactory' ✤ Run a job for loading and caching tables in DataFrames
 curl -d "" '127.0.0.1:8090/jobs? appName=test&classPath=spark.jobserver.SqlLoaderJob&context=sql- context&sync=true' ✤ Supply a query to a Query Job. All queries are logged in database by Spark Job Server.
 curl -d ‘sql=“SELECT count(*) FROM footable”’ '127.0.0.1:8090/jobs? appName=test&classPath=spark.jobserver.SqlQueryJob&context=sql- context&sync=true' 54
  • 55. Example: Combining Streaming And Spark SQL 55 SparkSQLStreamingContext Kafka Streaming Job SQL Query Job DataFrames Spark Job Server SQL Query
  • 56. SparkSQLStreamingJob 56 trait SparkSqlStreamingJob extends SparkJobBase { type C = SQLStreamingContext } class SQLStreamingContext(c: SparkContext) { val streamingContext = new StreamingContext(c, ...) val sqlContext = new SQLContext(c) } Now you have access to both StreamingContext and SQLContext, and it can be shared across jobs!
  • 57. SparkSQLStreamingContext 57 To start this context:
 curl -d "" “localhost:8090/contexts/stream_sqltest?context- factory=com.abc.SQLStreamingContextFactory" class SQLStreamingContextFactory extends SparkContextFactory { import SparkJobUtils._ type C = SQLStreamingContext with ContextLike def makeContext(config: Config, contextConfig: Config, contextName: String): C = { val batchInterval = contextConfig.getInt("batch_interval") val conf = configToSparkConf(config, contextConfig, contextName) new SQLStreamingContext(new SparkContext(conf), Seconds(batchInterval)) with ContextLike { def sparkContext: SparkContext = this.streamingContext.sparkContext def isValidJob(job: SparkJobBase): Boolean = job.isInstanceOf[SparkSqlStreamingJob] // Stop the streaming context, but not the SparkContext so that it can be re-used // to create another streaming context if required: def stop() { this.streamingContext.stop(false) } } } }
  • 59. Future Plans • PR: Forked JVMs for supporting many concurrent contexts • True HA operation • Swagger API documentation 59
  • 60. HA for Job Server Job Server 1 Job Server 2 Active Job Context Gossip Load balancer 60 Database GET /jobs/<id>
  • 61. HA and Hot Failover for Jobs Job Server 1 Job Server 2 Active Job Context HDFS Standby Job Context Gossip Checkpoint 61
  • 62. Thanks for your contributions! • All of these were community contributed: –HTTPS and Auth –saving and retrieving job configuration –forked JVM per context • Your contributions are very welcome on Github! 62
  • 63. 63 And Everybody is Hiring!! Thank you!
  • 64. WhyWe Needed a Job Server • Our vision for Spark is as a multi-team big data service • What gets repeated by every team: • Bastion box for running Hadoop/Spark jobs • Deploys and process monitoring • Tracking and serializing job status, progress, and job results • Job validation • No easy way to kill jobs • Polyglot technology stack - Ruby scripts run jobs, Go services
  • 66. Completely Async Design ✤ http://spray.io - probably the fastest JVM HTTP microframework ✤ Akka Actor based, non blocking ✤ Futures used to manage individual jobs. (Note that Spark is using Scala futures to manage job stages now) ✤ Single JVM for now, but easy to distribute later via remote Actors / Akka Cluster
  • 67. Async Actor Flow Spray web API Request actor Local Supervisor Job Manager Job 1 Future Job 2 Future Job Status Actor Job Result Actor
  • 68. Message flow fully documented
  • 70. Metadata Store ✤ JarInfo, JobInfo, ConfigInfo ✤ JobSqlDAO. Store metadata to SQL database by JDBC interface. ✤ Easily configured by spark.sqldao.jdbc.url ✤ jdbc:mysql://dbserver:3306/jobserverdb ✤ Multiple Job Servers can share the same MySQL. ✤ Jars uploaded once but accessible by all servers. ✤ The default will be JobSqlDAO and H2. ✤ Single H2 DB file. Serialization and deserialization are handled by H2.
  • 71. Deployment and Metrics ✤ spark-jobserver repo comes with a full suite of tests and deploy scripts: ✤ server_deploy.sh for regular server pushes ✤ server_package.sh for Mesos and Chronos .tar.gz ✤ /metricz route for codahale-metrics monitoring ✤ /healthz route for health check
  • 72. Challenges and Lessons • Spark is based around contexts - we need a Job Server oriented around logical jobs • Running multiple SparkContexts in the same process • Better long term solution is forked JVM per SparkContext • Workaround: spark.driver.allowMultipleContexts = true • Dynamic jar and class loading is tricky • Manage threads carefully - each context uses lots of threads