SlideShare a Scribd company logo
1 of 48
Download to read offline
Recipes for Running Spark
Streaming Apps in Production
Tathagata “TD” Das
Spark Summit 2015
@tathadas
Spark Streaming
Scalable, fault-tolerant stream processing system
File systems
Databases
Dashboards
Flume
Kinesis
HDFS/S3
Kafka
Twitter
High-level API
joins, windows, …
often 5x less code
Fault-tolerant
Exactly-once semantics,
even for stateful ops
Integration
Integrates with MLlib, SQL,
DataFrames, GraphX
Spark Streaming
Receivers receive data streams and chop them up into batches
Spark processes the batches and pushes out the results
3
data streams
receivers
batches results
Word Count with Kafka
val	
  context	
  =	
  new	
  StreamingContext(conf,	
  Seconds(1))	
  
val	
  lines	
  =	
  KafkaUtils.createStream(context,	
  ...)	
  
4
entry point of streaming
functionality
create DStream
from Kafka data
Word Count with Kafka
val	
  context	
  =	
  new	
  StreamingContext(conf,	
  Seconds(1))	
  
val	
  lines	
  =	
  KafkaUtils.createStream(context,	
  ...)	
  
val	
  words	
  =	
  lines.flatMap(_.split("	
  "))	
  
5
split lines into words
Word Count with Kafka
val	
  context	
  =	
  new	
  StreamingContext(conf,	
  Seconds(1))	
  
val	
  lines	
  =	
  KafkaUtils.createStream(context,	
  ...)	
  
val	
  words	
  =	
  lines.flatMap(_.split("	
  "))	
  
val	
  wordCounts	
  =	
  words.map(x	
  =>	
  (x,	
  1))	
  
	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  .reduceByKey(_	
  +	
  _)	
  
wordCounts.print()	
  
context.start()	
  
6
print some counts on screen
count the words
start receiving and
transforming the data
Word Count with Kafka
object	
  WordCount	
  {	
  
	
  	
  def	
  main(args:	
  Array[String])	
  {	
  
	
  	
  	
  	
  val	
  context	
  =	
  new	
  StreamingContext(new	
  SparkConf(),	
  Seconds(1))	
  
	
  	
  	
  	
  val	
  lines	
  =	
  KafkaUtils.createStream(context,	
  ...)	
  
	
  	
  	
  	
  val	
  words	
  =	
  lines.flatMap(_.split("	
  "))	
  
	
  	
  	
  	
  val	
  wordCounts	
  =	
  words.map(x	
  =>	
  (x,1)).reduceByKey(_	
  +	
  _)	
  
	
  	
  	
  	
  wordCounts.print()	
  
	
  	
  	
  	
  context.start()	
  
	
  	
  	
  	
  context.awaitTermination()	
  
	
  	
  }	
  
}	
  
7
Got it working on a small
Spark cluster on little data
What’s next??
How to get it production-ready?
Fault-tolerance and Semantics
Performance and Stability
Monitoring and Upgrading
8
9
Deeper View of
Spark Streaming
Any Spark Application
10
Driver
User code runs in
the driver process
YARN / Mesos /
Spark Standalone
cluster
Tasks sent to
executors for
processing data
Executor
Executor
Executor
Driver launches
executors in
cluster
Spark Streaming Application: Receive data
11
Executor
Executor
Driver runs
receivers as long
running tasks Receiver Data stream
Driver	
  
object	
  WordCount	
  {	
  
	
  	
  def	
  main(args:	
  Array[String])	
  {	
  
	
  	
  	
  	
  val	
  context	
  =	
  new	
  StreamingContext(...)	
  
	
  	
  	
  	
  val	
  lines	
  =	
  KafkaUtils.createStream(...)	
  
	
  	
  	
  	
  val	
  words	
  =	
  lines.flatMap(_.split("	
  "))	
  
	
  	
  	
  	
  val	
  wordCounts	
  =	
  words.map(x	
  =>	
  (x,1))	
  
	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  .reduceByKey(_	
  +	
  _)	
  
	
  	
  	
  	
  wordCounts.print()	
  
	
  	
  	
  	
  context.start()	
  
	
  	
  	
  	
  context.awaitTermination()	
  
	
  	
  }	
  
}	
  
Receiver divides
stream into blocks and
keeps in memory
Data Blocks	
  
Blocks also
replicated to
another executor Data Blocks	
  
Spark Streaming Application: Process data
12
Executor
Executor
Receiver
Data Blocks	
  
Data Blocks	
  
Data
store
Every batch
interval, driver
launches tasks to
process the blocks
Driver	
  
object	
  WordCount	
  {	
  
	
  	
  def	
  main(args:	
  Array[String])	
  {	
  
	
  	
  	
  	
  val	
  context	
  =	
  new	
  StreamingContext(...)	
  
	
  	
  	
  	
  val	
  lines	
  =	
  KafkaUtils.createStream(...)	
  
	
  	
  	
  	
  val	
  words	
  =	
  lines.flatMap(_.split("	
  "))	
  
	
  	
  	
  	
  val	
  wordCounts	
  =	
  words.map(x	
  =>	
  (x,1))	
  
	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  	
  .reduceByKey(_	
  +	
  _)	
  
	
  	
  	
  	
  wordCounts.print()	
  
	
  	
  	
  	
  context.start()	
  
	
  	
  	
  	
  context.awaitTermination()	
  
	
  	
  }	
  
}	
  
Fault-tolerance and Semantics
Performance and Stability
Monitoring and upgrading
13
Failures? Why care?
Many streaming applications need zero data loss
guarantees despite any kind of failures in the system
At least once guarantee – every record processed at least once
Exactly once guarantee – every record processed exactly once
Different kinds of failures – executor and driver
Some failures and guarantee requirements need
additional configurations and setups
14
Executor
Receiver
Data Blocks	
  
What if an executor fails?
Tasks and receivers restarted by Spark
automatically, no config needed
15
Executor
Failed Ex.
Receiver
Blocks	
  
Blocks	
  
Driver
If executor fails,
receiver is lost and
all blocks are lost
Receiver
Receiver
restarted
Tasks restarted
on block replicas
What if the driver fails?
16
Executor
Blocks	
  How do we
recover?
When the driver
fails, all the
executors fail
All computation,
all received
blocks are lost
Executor
Receiver
Blocks	
  
Failed Ex.
Receiver
Blocks	
  
Failed
Executor
Blocks	
  
Driver
Failed
Driver
Recovering Driver with Checkpointing
DStream Checkpointing:
Periodically save the DAG of DStreams to
fault-tolerant storage
17
Executor
Blocks	
  
Executor
Receiver
Blocks	
  
Active
Driver
Checkpoint info
to HDFS / S3
Recovering Driver w/ DStream Checkpointing
18
Failed driver can be restarted
from checkpoint information
Failed
Driver
Restarted
Driver
New
Executor
New
Executor
Receiver
New executors
launched and
receivers
restarted
DStream Checkpointing:
Periodically save the DAG of DStreams to
fault-tolerant storage
Recovering Driver w/ DStream Checkpointing
1.  Configure automatic driver restart
All cluster managers support this
2.  Set a checkpoint directory in a HDFS-compatible file
system
	
  streamingContext.checkpoint(hdfsDirectory)	
  
3.  Slightly restructure of the code to use checkpoints for
recovery
!
19
Configurating Automatic Driver Restart
Spark Standalone – Use spark-submit with “cluster” mode and “--supervise”
See http://spark.apache.org/docs/latest/spark-standalone.html
YARN – Use spark-submit in “cluster” mode
See YARN config “yarn.resourcemanager.am.max-attempts”
Mesos – Marathon can restart Mesos applications
20
Restructuring code for Checkpointing
21
val	
  context	
  =	
  new	
  StreamingContext(...)	
  
val	
  lines	
  =	
  KafkaUtils.createStream(...)	
  
val	
  words	
  =	
  lines.flatMap(...)	
  
...	
  
context.start()	
  
Create
+
Setup
Start
def	
  creatingFunc():	
  StreamingContext	
  =	
  {	
  	
  
	
  	
  	
  val	
  context	
  =	
  new	
  StreamingContext(...)	
  	
  	
  
	
  	
  	
  val	
  lines	
  =	
  KafkaUtils.createStream(...)	
  
	
  	
  	
  val	
  words	
  =	
  lines.flatMap(...)	
  
	
  	
  	
  ...	
  
	
  	
  	
  context.checkpoint(hdfsDir)	
  
}	
  
Put all setup code into a function that returns a new StreamingContext
Get context setup from HDFS dir OR create a new one with the function
val	
  context	
  =	
  
StreamingContext.getOrCreate(	
  
	
  	
  hdfsDir,	
  creatingFunc)	
  
context.start()	
  
Restructuring code for Checkpointing
StreamingContext.getOrCreate():
If HDFS directory has checkpoint info
recover context from info
else
call creatingFunc() to create
and setup a new context
Restarted process can figure out whether
to recover using checkpoint info or not
22
def	
  creatingFunc():	
  StreamingContext	
  =	
  {	
  	
  
	
  	
  	
  val	
  context	
  =	
  new	
  StreamingContext(...)	
  	
  	
  
	
  	
  	
  val	
  lines	
  =	
  KafkaUtils.createStream(...)	
  
	
  	
  	
  val	
  words	
  =	
  lines.flatMap(...)	
  
	
  	
  	
  ...	
  
	
  	
  	
  context.checkpoint(hdfsDir)	
  
}	
  
val	
  context	
  =	
  
StreamingContext.getOrCreate(	
  
	
  	
  hdfsDir,	
  creatingFunc)	
  
context.start()	
  
Received blocks lost on Restart!
23
Failed
Driver
Restarted
Driver
New
Executor
New Ex.
Receiver
No Blocks	
  
In-memory blocks of
buffered data are
lost on driver restart
Recovering data with Write Ahead Logs
Write Ahead Log (WAL): Synchronously save
received data to fault-tolerant storage
24
Executor
Blocks saved
to HDFS
Executor
Receiver
Blocks	
  
Active
Driver
Data stream
Recovering data with Write Ahead Logs
25
Failed
Driver
Restarted
Driver
New
Executor
New Ex.
Receiver
Blocks	
  
Blocks recovered
from Write Ahead Log
Write Ahead Log (WAL): Synchronously save
received data to fault-tolerant storage
Recovering data with Write Ahead Logs
1.  Enable checkpointing, logs written in checkpoint directory
2.  Enabled WAL in SparkConf configuration
	
  	
  	
  	
  	
  sparkConf.set("spark.streaming.receiver.writeAheadLog.enable",	
  "true")	
  
3.  Receiver should also be reliable
Acknowledge source only after data saved to WAL
Unacked data will be replayed from source by restarted receiver
4.  Disable in-memory replication (already replicated by HDFS)
	
  Use StorageLevel.MEMORY_AND_DISK_SER for input DStreams
26
RDD Checkpointing
Stateful stream processing can lead to long RDD lineages
Long lineage = bad for fault-tolerance, too much recomputation
RDD checkpointing saves RDD data to the fault-tolerant
storage to limit lineage and recomputation
More: http://spark.apache.org/docs/latest/streaming-programming-guide.html#checkpointing
27
Fault-tolerance Semantics
28
Zero data loss = every stage processes each
event at least once despite any failure
Sources
Transforming
Sinks
Outputting
Receiving
Fault-tolerance Semantics
29
Sources
Transforming
Sinks
Outputting
Receiving
Exactly once, as long as received data is not lost
At least once, w/ Checkpointing + WAL + Reliable receiversReceiving
Outputting
Exactly once, if outputs are idempotent or transactional
End-to-end semantics:
At-least once
Fault-tolerance Semantics
30
Exactly once receiving with new Kafka Direct approach
Treats Kafka like a replicated log, reads it like a file
Does not use receivers
No need to create multiple DStreams and union them
No need to enable Write Ahead Logs
	
  
	
  val	
  directKafkaStream	
  =	
  KafkaUtils.createDirectStream(...)	
  
	
  
https://databricks.com/blog/2015/03/30/improvements-to-kafka-integration-of-spark-streaming.html
http://spark.apache.org/docs/latest/streaming-kafka-integration.html
Sources
Transforming
Sinks
Outputting
Receiving
Fault-tolerance Semantics
31
Exactly once receiving with new Kafka Direct approach
Sources
Transforming
Sinks
Outputting
Receiving
Exactly once, as long as received data is not lost
Exactly once, if outputs are idempotent or transactional
End-to-end semantics:
Exactly once!
32
http://techblog.netflix.com/2015/03/can-spark-streaming-survive-chaos-monkey.html
Fault-tolerance and Semantics
Performance and Stability
Monitoring and Upgrading
33
Achieving High Throughput
34
High throughput achieved by sufficient
parallelism at all stages of the pipeline
Sources
Transforming
Sinks
Outputting
Receiving
Scaling the Receivers
35
Sources
Transforming
Sinks
Outputting
Receiving
Sources must be configured with parallel data streams
#partitions in Kafka topics, #shards in Kinesis streams, …
Streaming app should have multiple receivers that
receive the data streams in parallel
Multiple input DStreams, each running a receiver
Can be unioned together to create one DStream
	
  val	
  kafkaStream1	
  =	
  KafkaUtils.createStream(...)	
  
	
  val	
  kafkaStream2	
  =	
  KafkaUtils.createStream(...)	
  
	
  val	
  unionedStream	
  =	
  kafkaStream1.union(kafkaStream2)	
  
	
  
Scaling the Receivers
36
Sources
Transforming
Sinks
Outputting
Receiving
Sufficient number of executors to run all the receivers
Absolute necessity: #cores > #receivers
Good rule of thumb: #executors > #receivers, so that no more
than 1 receiver per executor, and network is not shared
between receivers
Kafka Direct approach does not use receivers
Automatically parallelizes data reading across executors
Parallelism = # Kafka partitions
Stability in Processing
37
Sources
Transforming
Sinks
Outputting
Receiving For stability, must process data as fast as it is received
Must ensure avg batch processing times < batch interval
Previous batch is done by the time next batch is received
Otherwise, new batches keeps queueing up waiting for
previous batches to finish, scheduling delay goes up
Reducing Batch Processing Times
38
Sources
Transforming
Sinks
Outputting
Receiving
More receivers!
Executor running receivers do lot of the processing
Repartition the received data to explicitly distribute load
unionedStream.repartition(40)	
  
Set #partitions in shuffles, make sure its large enough
transformedStream.reduceByKey(reduceFunc,	
  40)	
  
Get more executors and cores!
Reducing Batch Processing Times
39
Sources
Transforming
Sinks
Outputting
Receiving
Use Kryo serialization to serialization costs
Register classes for best performance
See configurations spark.kryo.*
http://spark.apache.org/docs/latest/configuration.html#compression-and-serialization
Larger batch durations improve stability
More data aggregated together, amortized cost of shuffle
Limit ingestion rate to handle data surges
See configurations spark.streaming.*maxRate*
http://spark.apache.org/docs/latest/configuration.html#spark-streaming
Speeding up Output Operations
40
Sources
Transforming
Sinks
Outputting
Receiving
Write to data stores efficiently
dataRDD.foreach	
  {	
  event	
  =>	
  
	
  	
  //	
  open	
  connection	
  
	
  	
  //	
  insert	
  single	
  event	
  
	
  	
  //	
  close	
  connection	
  
}	
  
foreach: inefficient
dataRDD.foreachPartition	
  {	
  partition	
  =>	
  
	
  	
  //	
  open	
  connection	
  
	
  	
  //	
  insert	
  all	
  events	
  in	
  partition	
  
	
  	
  //	
  close	
  connection	
  
}	
  
foreachPartition: efficient
dataRDD.foreachPartition	
  {	
  partition	
  =>	
  
	
  	
  //	
  initialize	
  pool	
  or	
  get	
  open	
  connection	
  from	
  pool	
  in	
  executor	
  
	
  	
  //	
  insert	
  all	
  events	
  in	
  partition	
  
	
  	
  //	
  return	
  connection	
  to	
  pool	
  
}	
  
foreachPartition + connection pool: more efficient
Fault-tolerance and Semantics
Performance and Stability
Monitoring and Upgrading
41
Streaming in Spark Web UI
Stats over last 1000 batches
New in Spark 1.4
42
For stability
Scheduling delay should be approx 0
Processing Time approx < batch interval
Streaming in Spark Web UI
Details of individual batches
43
Details of Spark jobs run in a batch
Operational Monitoring
Streaming app stats published through Codahale metrics
Ganglia sink, Graphite sink, custom Codahale metrics sinks
Can see long term trends, across hours and days
Configure the metrics using $SPARK_HOME/conf/metrics.properties	
  
Need to compile Spark with Ganglia LGPL profile for Ganglia support
(see http://spark.apache.org/docs/latest/monitoring.html#metrics)
44
Programmatic Monitoring
StreamingListener – Developer interface to get internal events
onBatchSubmitted, onBatchStarted, onBatchCompleted,
onReceiverStarted, onReceiverStopped, onReceiverError
Take a look at StreamingJobProgressListener (private class) for
inspiration
45
Upgrading Apps
1.  Shutdown your current streaming app gracefully
Will process all data before shutting down cleanly
streamingContext.stop(stopGracefully	
  =	
  true)	
  
2.  Update app code and start it again
Cannot upgrade from previous checkpoints if code
changes or Spark version changes
	
  
46
Much to say I have ... but time I have not
Memory and GC tuning
Using SQLContext
DStream.transform operation
…
Refer to online guide
http://spark.apache.org/docs/latest/streaming-programming-guide.html
47
Thank you
May the stream be with you

More Related Content

What's hot

Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015Evan Chan
 
Stream Processing using Apache Spark and Apache Kafka
Stream Processing using Apache Spark and Apache KafkaStream Processing using Apache Spark and Apache Kafka
Stream Processing using Apache Spark and Apache KafkaAbhinav Singh
 
Getting Started Running Apache Spark on Apache Mesos
Getting Started Running Apache Spark on Apache MesosGetting Started Running Apache Spark on Apache Mesos
Getting Started Running Apache Spark on Apache MesosPaco Nathan
 
Real time data viz with Spark Streaming, Kafka and D3.js
Real time data viz with Spark Streaming, Kafka and D3.jsReal time data viz with Spark Streaming, Kafka and D3.js
Real time data viz with Spark Streaming, Kafka and D3.jsBen Laird
 
Strata NYC 2015: What's new in Spark Streaming
Strata NYC 2015: What's new in Spark StreamingStrata NYC 2015: What's new in Spark Streaming
Strata NYC 2015: What's new in Spark StreamingDatabricks
 
Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...
Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...
Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...Helena Edelson
 
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 meetupRafal Kwasny
 
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 SparkRahul Jain
 
SMACK Stack 1.1
SMACK Stack 1.1SMACK Stack 1.1
SMACK Stack 1.1Joe Stein
 
Alpine academy apache spark series #1 introduction to cluster computing wit...
Alpine academy apache spark series #1   introduction to cluster computing wit...Alpine academy apache spark series #1   introduction to cluster computing wit...
Alpine academy apache spark series #1 introduction to cluster computing wit...Holden Karau
 
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...Lucidworks
 
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)Helena Edelson
 
Cassandra and SparkSQL: You Don't Need Functional Programming for Fun with Ru...
Cassandra and SparkSQL: You Don't Need Functional Programming for Fun with Ru...Cassandra and SparkSQL: You Don't Need Functional Programming for Fun with Ru...
Cassandra and SparkSQL: You Don't Need Functional Programming for Fun with Ru...Databricks
 
Real-time personal trainer on the SMACK stack
Real-time personal trainer on the SMACK stackReal-time personal trainer on the SMACK stack
Real-time personal trainer on the SMACK stackAnirvan Chakraborty
 
Spark Summit EU talk by Miklos Christine paddling up the stream
Spark Summit EU talk by Miklos Christine paddling up the streamSpark Summit EU talk by Miklos Christine paddling up the stream
Spark Summit EU talk by Miklos Christine paddling up the streamSpark Summit
 
NoLambda: Combining Streaming, Ad-Hoc, Machine Learning and Batch Analysis
NoLambda: Combining Streaming, Ad-Hoc, Machine Learning and Batch AnalysisNoLambda: Combining Streaming, Ad-Hoc, Machine Learning and Batch Analysis
NoLambda: Combining Streaming, Ad-Hoc, Machine Learning and Batch AnalysisHelena Edelson
 
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17spark-project
 
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 ServiceEvan Chan
 
Operational Tips for Deploying Spark by Miklos Christine
Operational Tips for Deploying Spark by Miklos ChristineOperational Tips for Deploying Spark by Miklos Christine
Operational Tips for Deploying Spark by Miklos ChristineSpark Summit
 

What's hot (20)

Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015
 
Stream Processing using Apache Spark and Apache Kafka
Stream Processing using Apache Spark and Apache KafkaStream Processing using Apache Spark and Apache Kafka
Stream Processing using Apache Spark and Apache Kafka
 
Getting Started Running Apache Spark on Apache Mesos
Getting Started Running Apache Spark on Apache MesosGetting Started Running Apache Spark on Apache Mesos
Getting Started Running Apache Spark on Apache Mesos
 
Real time data viz with Spark Streaming, Kafka and D3.js
Real time data viz with Spark Streaming, Kafka and D3.jsReal time data viz with Spark Streaming, Kafka and D3.js
Real time data viz with Spark Streaming, Kafka and D3.js
 
Strata NYC 2015: What's new in Spark Streaming
Strata NYC 2015: What's new in Spark StreamingStrata NYC 2015: What's new in Spark Streaming
Strata NYC 2015: What's new in Spark Streaming
 
Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...
Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...
Delivering Meaning In Near-Real Time At High Velocity In Massive Scale with A...
 
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
 
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
 
SMACK Stack 1.1
SMACK Stack 1.1SMACK Stack 1.1
SMACK Stack 1.1
 
Alpine academy apache spark series #1 introduction to cluster computing wit...
Alpine academy apache spark series #1   introduction to cluster computing wit...Alpine academy apache spark series #1   introduction to cluster computing wit...
Alpine academy apache spark series #1 introduction to cluster computing wit...
 
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...
Near Real Time Indexing Kafka Messages into Apache Blur: Presented by Dibyend...
 
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
 
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
 
Cassandra and SparkSQL: You Don't Need Functional Programming for Fun with Ru...
Cassandra and SparkSQL: You Don't Need Functional Programming for Fun with Ru...Cassandra and SparkSQL: You Don't Need Functional Programming for Fun with Ru...
Cassandra and SparkSQL: You Don't Need Functional Programming for Fun with Ru...
 
Real-time personal trainer on the SMACK stack
Real-time personal trainer on the SMACK stackReal-time personal trainer on the SMACK stack
Real-time personal trainer on the SMACK stack
 
Spark Summit EU talk by Miklos Christine paddling up the stream
Spark Summit EU talk by Miklos Christine paddling up the streamSpark Summit EU talk by Miklos Christine paddling up the stream
Spark Summit EU talk by Miklos Christine paddling up the stream
 
NoLambda: Combining Streaming, Ad-Hoc, Machine Learning and Batch Analysis
NoLambda: Combining Streaming, Ad-Hoc, Machine Learning and Batch AnalysisNoLambda: Combining Streaming, Ad-Hoc, Machine Learning and Batch Analysis
NoLambda: Combining Streaming, Ad-Hoc, Machine Learning and Batch Analysis
 
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
 
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
 
Operational Tips for Deploying Spark by Miklos Christine
Operational Tips for Deploying Spark by Miklos ChristineOperational Tips for Deploying Spark by Miklos Christine
Operational Tips for Deploying Spark by Miklos Christine
 

Viewers also liked

Viewers also liked (10)

24 June 2015: Working with CDE
24 June 2015: Working with CDE24 June 2015: Working with CDE
24 June 2015: Working with CDE
 
Petabyte Scale Anomaly Detection Using R & Spark by Sridhar Alla and Kiran Mu...
Petabyte Scale Anomaly Detection Using R & Spark by Sridhar Alla and Kiran Mu...Petabyte Scale Anomaly Detection Using R & Spark by Sridhar Alla and Kiran Mu...
Petabyte Scale Anomaly Detection Using R & Spark by Sridhar Alla and Kiran Mu...
 
OSINT for Attack and Defense
OSINT for Attack and DefenseOSINT for Attack and Defense
OSINT for Attack and Defense
 
OSINT 2.0 - Past, present and future
OSINT 2.0  - Past, present and futureOSINT 2.0  - Past, present and future
OSINT 2.0 - Past, present and future
 
OSINT - Open Source Intelligence
OSINT - Open Source IntelligenceOSINT - Open Source Intelligence
OSINT - Open Source Intelligence
 
Relationship Extraction from Unstructured Text-Based on Stanford NLP with Spa...
Relationship Extraction from Unstructured Text-Based on Stanford NLP with Spa...Relationship Extraction from Unstructured Text-Based on Stanford NLP with Spa...
Relationship Extraction from Unstructured Text-Based on Stanford NLP with Spa...
 
Open Source Intelligence (OSINT)
Open Source Intelligence (OSINT)Open Source Intelligence (OSINT)
Open Source Intelligence (OSINT)
 
2017 Digital Yearbook
2017 Digital Yearbook2017 Digital Yearbook
2017 Digital Yearbook
 
Digital in 2017 Global Overview
Digital in 2017 Global OverviewDigital in 2017 Global Overview
Digital in 2017 Global Overview
 
Global Digital Statshot Q3 2017
Global Digital Statshot Q3 2017Global Digital Statshot Q3 2017
Global Digital Statshot Q3 2017
 

Similar to Recipes for Running Spark Streaming Applications in Production-(Tathagata Das, Databricks)

Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...DataWorks Summit
 
Four Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and DatabricksFour Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and DatabricksLegacy Typesafe (now Lightbend)
 
Learning spark ch10 - Spark Streaming
Learning spark ch10 - Spark StreamingLearning spark ch10 - Spark Streaming
Learning spark ch10 - Spark Streamingphanleson
 
Productionizing your Streaming Jobs
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming JobsDatabricks
 
Flink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseFlink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseKostas Tzoumas
 
Spark streaming with kafka
Spark streaming with kafkaSpark streaming with kafka
Spark streaming with kafkaDori Waldman
 
Productionalizing spark streaming applications
Productionalizing spark streaming applicationsProductionalizing spark streaming applications
Productionalizing spark streaming applicationsRobert Sanders
 
Apache Flink Overview at SF Spark and Friends
Apache Flink Overview at SF Spark and FriendsApache Flink Overview at SF Spark and Friends
Apache Flink Overview at SF Spark and FriendsStephan Ewen
 
First Flink Bay Area meetup
First Flink Bay Area meetupFirst Flink Bay Area meetup
First Flink Bay Area meetupKostas Tzoumas
 
Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一
Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一
Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一scalaconfjp
 
Spark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics RevisedSpark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics RevisedMichael Spector
 
Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...
Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...
Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...Codemotion
 
Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...
Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...
Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...Codemotion
 
Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)Stephan Ewen
 
Spark streaming state of the union
Spark streaming state of the unionSpark streaming state of the union
Spark streaming state of the unionDatabricks
 
January 2016 Flink Community Update & Roadmap 2016
January 2016 Flink Community Update & Roadmap 2016January 2016 Flink Community Update & Roadmap 2016
January 2016 Flink Community Update & Roadmap 2016Robert Metzger
 
From Zero to Stream Processing
From Zero to Stream ProcessingFrom Zero to Stream Processing
From Zero to Stream ProcessingEventador
 

Similar to Recipes for Running Spark Streaming Applications in Production-(Tathagata Das, Databricks) (20)

Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
 
Four Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and DatabricksFour Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and Databricks
 
Learning spark ch10 - Spark Streaming
Learning spark ch10 - Spark StreamingLearning spark ch10 - Spark Streaming
Learning spark ch10 - Spark Streaming
 
Productionizing your Streaming Jobs
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming Jobs
 
Flink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseFlink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San Jose
 
Spark streaming with kafka
Spark streaming with kafkaSpark streaming with kafka
Spark streaming with kafka
 
Productionalizing spark streaming applications
Productionalizing spark streaming applicationsProductionalizing spark streaming applications
Productionalizing spark streaming applications
 
Apache Flink Overview at SF Spark and Friends
Apache Flink Overview at SF Spark and FriendsApache Flink Overview at SF Spark and Friends
Apache Flink Overview at SF Spark and Friends
 
First Flink Bay Area meetup
First Flink Bay Area meetupFirst Flink Bay Area meetup
First Flink Bay Area meetup
 
Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一
Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一
Building a Unified Data Pipline in Spark / Apache Sparkを用いたBig Dataパイプラインの統一
 
Spark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics RevisedSpark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics Revised
 
Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...
Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...
Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...
 
Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...
Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...
Jörg Schad - Hybrid Cloud (Kubernetes, Spark, HDFS, …)-as-a-Service - Codemot...
 
Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)
 
Sparkstreaming
SparkstreamingSparkstreaming
Sparkstreaming
 
20170126 big data processing
20170126 big data processing20170126 big data processing
20170126 big data processing
 
Spark streaming state of the union
Spark streaming state of the unionSpark streaming state of the union
Spark streaming state of the union
 
January 2016 Flink Community Update & Roadmap 2016
January 2016 Flink Community Update & Roadmap 2016January 2016 Flink Community Update & Roadmap 2016
January 2016 Flink Community Update & Roadmap 2016
 
Apache Flink Deep Dive
Apache Flink Deep DiveApache Flink Deep Dive
Apache Flink Deep Dive
 
From Zero to Stream Processing
From Zero to Stream ProcessingFrom Zero to Stream Processing
From Zero to Stream Processing
 

More from Spark Summit

FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang
FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang
FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang Spark Summit
 
VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...
VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...
VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...Spark Summit
 
Apache Spark Structured Streaming Helps Smart Manufacturing with Xiaochang Wu
Apache Spark Structured Streaming Helps Smart Manufacturing with  Xiaochang WuApache Spark Structured Streaming Helps Smart Manufacturing with  Xiaochang Wu
Apache Spark Structured Streaming Helps Smart Manufacturing with Xiaochang WuSpark Summit
 
Improving Traffic Prediction Using Weather Data with Ramya Raghavendra
Improving Traffic Prediction Using Weather Data  with Ramya RaghavendraImproving Traffic Prediction Using Weather Data  with Ramya Raghavendra
Improving Traffic Prediction Using Weather Data with Ramya RaghavendraSpark Summit
 
A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...
A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...
A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...Spark Summit
 
No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...
No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...
No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...Spark Summit
 
Apache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim DowlingApache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim DowlingSpark Summit
 
Apache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim DowlingApache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim DowlingSpark Summit
 
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...Spark Summit
 
Next CERN Accelerator Logging Service with Jakub Wozniak
Next CERN Accelerator Logging Service with Jakub WozniakNext CERN Accelerator Logging Service with Jakub Wozniak
Next CERN Accelerator Logging Service with Jakub WozniakSpark Summit
 
Powering a Startup with Apache Spark with Kevin Kim
Powering a Startup with Apache Spark with Kevin KimPowering a Startup with Apache Spark with Kevin Kim
Powering a Startup with Apache Spark with Kevin KimSpark Summit
 
Improving Traffic Prediction Using Weather Datawith Ramya Raghavendra
Improving Traffic Prediction Using Weather Datawith Ramya RaghavendraImproving Traffic Prediction Using Weather Datawith Ramya Raghavendra
Improving Traffic Prediction Using Weather Datawith Ramya RaghavendraSpark Summit
 
Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...
Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...
Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...Spark Summit
 
How Nielsen Utilized Databricks for Large-Scale Research and Development with...
How Nielsen Utilized Databricks for Large-Scale Research and Development with...How Nielsen Utilized Databricks for Large-Scale Research and Development with...
How Nielsen Utilized Databricks for Large-Scale Research and Development with...Spark Summit
 
Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...
Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...
Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...Spark Summit
 
Goal Based Data Production with Sim Simeonov
Goal Based Data Production with Sim SimeonovGoal Based Data Production with Sim Simeonov
Goal Based Data Production with Sim SimeonovSpark Summit
 
Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...
Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...
Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...Spark Summit
 
Getting Ready to Use Redis with Apache Spark with Dvir Volk
Getting Ready to Use Redis with Apache Spark with Dvir VolkGetting Ready to Use Redis with Apache Spark with Dvir Volk
Getting Ready to Use Redis with Apache Spark with Dvir VolkSpark Summit
 
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...Spark Summit
 
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...Spark Summit
 

More from Spark Summit (20)

FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang
FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang
FPGA-Based Acceleration Architecture for Spark SQL Qi Xie and Quanfu Wang
 
VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...
VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...
VEGAS: The Missing Matplotlib for Scala/Apache Spark with DB Tsai and Roger M...
 
Apache Spark Structured Streaming Helps Smart Manufacturing with Xiaochang Wu
Apache Spark Structured Streaming Helps Smart Manufacturing with  Xiaochang WuApache Spark Structured Streaming Helps Smart Manufacturing with  Xiaochang Wu
Apache Spark Structured Streaming Helps Smart Manufacturing with Xiaochang Wu
 
Improving Traffic Prediction Using Weather Data with Ramya Raghavendra
Improving Traffic Prediction Using Weather Data  with Ramya RaghavendraImproving Traffic Prediction Using Weather Data  with Ramya Raghavendra
Improving Traffic Prediction Using Weather Data with Ramya Raghavendra
 
A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...
A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...
A Tale of Two Graph Frameworks on Spark: GraphFrames and Tinkerpop OLAP Artem...
 
No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...
No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...
No More Cumbersomeness: Automatic Predictive Modeling on Apache Spark Marcin ...
 
Apache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim DowlingApache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim Dowling
 
Apache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim DowlingApache Spark and Tensorflow as a Service with Jim Dowling
Apache Spark and Tensorflow as a Service with Jim Dowling
 
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
MMLSpark: Lessons from Building a SparkML-Compatible Machine Learning Library...
 
Next CERN Accelerator Logging Service with Jakub Wozniak
Next CERN Accelerator Logging Service with Jakub WozniakNext CERN Accelerator Logging Service with Jakub Wozniak
Next CERN Accelerator Logging Service with Jakub Wozniak
 
Powering a Startup with Apache Spark with Kevin Kim
Powering a Startup with Apache Spark with Kevin KimPowering a Startup with Apache Spark with Kevin Kim
Powering a Startup with Apache Spark with Kevin Kim
 
Improving Traffic Prediction Using Weather Datawith Ramya Raghavendra
Improving Traffic Prediction Using Weather Datawith Ramya RaghavendraImproving Traffic Prediction Using Weather Datawith Ramya Raghavendra
Improving Traffic Prediction Using Weather Datawith Ramya Raghavendra
 
Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...
Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...
Hiding Apache Spark Complexity for Fast Prototyping of Big Data Applications—...
 
How Nielsen Utilized Databricks for Large-Scale Research and Development with...
How Nielsen Utilized Databricks for Large-Scale Research and Development with...How Nielsen Utilized Databricks for Large-Scale Research and Development with...
How Nielsen Utilized Databricks for Large-Scale Research and Development with...
 
Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...
Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...
Spline: Apache Spark Lineage not Only for the Banking Industry with Marek Nov...
 
Goal Based Data Production with Sim Simeonov
Goal Based Data Production with Sim SimeonovGoal Based Data Production with Sim Simeonov
Goal Based Data Production with Sim Simeonov
 
Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...
Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...
Preventing Revenue Leakage and Monitoring Distributed Systems with Machine Le...
 
Getting Ready to Use Redis with Apache Spark with Dvir Volk
Getting Ready to Use Redis with Apache Spark with Dvir VolkGetting Ready to Use Redis with Apache Spark with Dvir Volk
Getting Ready to Use Redis with Apache Spark with Dvir Volk
 
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
Deduplication and Author-Disambiguation of Streaming Records via Supervised M...
 
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...
MatFast: In-Memory Distributed Matrix Computation Processing and Optimization...
 

Recently uploaded

Networking Case Study prepared by teacher.pptx
Networking Case Study prepared by teacher.pptxNetworking Case Study prepared by teacher.pptx
Networking Case Study prepared by teacher.pptxHimangsuNath
 
Unveiling the Role of Social Media Suspect Investigators in Preventing Online...
Unveiling the Role of Social Media Suspect Investigators in Preventing Online...Unveiling the Role of Social Media Suspect Investigators in Preventing Online...
Unveiling the Role of Social Media Suspect Investigators in Preventing Online...Milind Agarwal
 
Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 2Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 217djon017
 
INTRODUCTION TO Natural language processing
INTRODUCTION TO Natural language processingINTRODUCTION TO Natural language processing
INTRODUCTION TO Natural language processingsocarem879
 
Real-Time AI Streaming - AI Max Princeton
Real-Time AI  Streaming - AI Max PrincetonReal-Time AI  Streaming - AI Max Princeton
Real-Time AI Streaming - AI Max PrincetonTimothy Spann
 
wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...
wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...
wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...KarteekMane1
 
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptxThe Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptxTasha Penwell
 
Learn How Data Science Changes Our World
Learn How Data Science Changes Our WorldLearn How Data Science Changes Our World
Learn How Data Science Changes Our WorldEduminds Learning
 
SMOTE and K-Fold Cross Validation-Presentation.pptx
SMOTE and K-Fold Cross Validation-Presentation.pptxSMOTE and K-Fold Cross Validation-Presentation.pptx
SMOTE and K-Fold Cross Validation-Presentation.pptxHaritikaChhatwal1
 
Conf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming PipelinesConf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming PipelinesTimothy Spann
 
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...Dr Arash Najmaei ( Phd., MBA, BSc)
 
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdfEnglish-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdfblazblazml
 
Principles and Practices of Data Visualization
Principles and Practices of Data VisualizationPrinciples and Practices of Data Visualization
Principles and Practices of Data VisualizationKianJazayeri1
 
Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)Cathrine Wilhelmsen
 
convolutional neural network and its applications.pdf
convolutional neural network and its applications.pdfconvolutional neural network and its applications.pdf
convolutional neural network and its applications.pdfSubhamKumar3239
 
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Boston Institute of Analytics
 
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...Amil Baba Dawood bangali
 
Defining Constituents, Data Vizzes and Telling a Data Story
Defining Constituents, Data Vizzes and Telling a Data StoryDefining Constituents, Data Vizzes and Telling a Data Story
Defining Constituents, Data Vizzes and Telling a Data StoryJeremy Anderson
 

Recently uploaded (20)

Networking Case Study prepared by teacher.pptx
Networking Case Study prepared by teacher.pptxNetworking Case Study prepared by teacher.pptx
Networking Case Study prepared by teacher.pptx
 
Data Analysis Project: Stroke Prediction
Data Analysis Project: Stroke PredictionData Analysis Project: Stroke Prediction
Data Analysis Project: Stroke Prediction
 
Unveiling the Role of Social Media Suspect Investigators in Preventing Online...
Unveiling the Role of Social Media Suspect Investigators in Preventing Online...Unveiling the Role of Social Media Suspect Investigators in Preventing Online...
Unveiling the Role of Social Media Suspect Investigators in Preventing Online...
 
Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 2Easter Eggs From Star Wars and in cars 1 and 2
Easter Eggs From Star Wars and in cars 1 and 2
 
INTRODUCTION TO Natural language processing
INTRODUCTION TO Natural language processingINTRODUCTION TO Natural language processing
INTRODUCTION TO Natural language processing
 
Real-Time AI Streaming - AI Max Princeton
Real-Time AI  Streaming - AI Max PrincetonReal-Time AI  Streaming - AI Max Princeton
Real-Time AI Streaming - AI Max Princeton
 
wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...
wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...
wepik-insightful-infographics-a-data-visualization-overview-20240401133220kwr...
 
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptxThe Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
 
Learn How Data Science Changes Our World
Learn How Data Science Changes Our WorldLearn How Data Science Changes Our World
Learn How Data Science Changes Our World
 
SMOTE and K-Fold Cross Validation-Presentation.pptx
SMOTE and K-Fold Cross Validation-Presentation.pptxSMOTE and K-Fold Cross Validation-Presentation.pptx
SMOTE and K-Fold Cross Validation-Presentation.pptx
 
Conf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming PipelinesConf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
 
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
 
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdfEnglish-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
English-8-Q4-W3-Synthesizing-Essential-Information-From-Various-Sources-1.pdf
 
Insurance Churn Prediction Data Analysis Project
Insurance Churn Prediction Data Analysis ProjectInsurance Churn Prediction Data Analysis Project
Insurance Churn Prediction Data Analysis Project
 
Principles and Practices of Data Visualization
Principles and Practices of Data VisualizationPrinciples and Practices of Data Visualization
Principles and Practices of Data Visualization
 
Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)
 
convolutional neural network and its applications.pdf
convolutional neural network and its applications.pdfconvolutional neural network and its applications.pdf
convolutional neural network and its applications.pdf
 
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
 
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
 
Defining Constituents, Data Vizzes and Telling a Data Story
Defining Constituents, Data Vizzes and Telling a Data StoryDefining Constituents, Data Vizzes and Telling a Data Story
Defining Constituents, Data Vizzes and Telling a Data Story
 

Recipes for Running Spark Streaming Applications in Production-(Tathagata Das, Databricks)

  • 1. Recipes for Running Spark Streaming Apps in Production Tathagata “TD” Das Spark Summit 2015 @tathadas
  • 2. Spark Streaming Scalable, fault-tolerant stream processing system File systems Databases Dashboards Flume Kinesis HDFS/S3 Kafka Twitter High-level API joins, windows, … often 5x less code Fault-tolerant Exactly-once semantics, even for stateful ops Integration Integrates with MLlib, SQL, DataFrames, GraphX
  • 3. Spark Streaming Receivers receive data streams and chop them up into batches Spark processes the batches and pushes out the results 3 data streams receivers batches results
  • 4. Word Count with Kafka val  context  =  new  StreamingContext(conf,  Seconds(1))   val  lines  =  KafkaUtils.createStream(context,  ...)   4 entry point of streaming functionality create DStream from Kafka data
  • 5. Word Count with Kafka val  context  =  new  StreamingContext(conf,  Seconds(1))   val  lines  =  KafkaUtils.createStream(context,  ...)   val  words  =  lines.flatMap(_.split("  "))   5 split lines into words
  • 6. Word Count with Kafka val  context  =  new  StreamingContext(conf,  Seconds(1))   val  lines  =  KafkaUtils.createStream(context,  ...)   val  words  =  lines.flatMap(_.split("  "))   val  wordCounts  =  words.map(x  =>  (x,  1))                                              .reduceByKey(_  +  _)   wordCounts.print()   context.start()   6 print some counts on screen count the words start receiving and transforming the data
  • 7. Word Count with Kafka object  WordCount  {      def  main(args:  Array[String])  {          val  context  =  new  StreamingContext(new  SparkConf(),  Seconds(1))          val  lines  =  KafkaUtils.createStream(context,  ...)          val  words  =  lines.flatMap(_.split("  "))          val  wordCounts  =  words.map(x  =>  (x,1)).reduceByKey(_  +  _)          wordCounts.print()          context.start()          context.awaitTermination()      }   }   7 Got it working on a small Spark cluster on little data What’s next??
  • 8. How to get it production-ready? Fault-tolerance and Semantics Performance and Stability Monitoring and Upgrading 8
  • 10. Any Spark Application 10 Driver User code runs in the driver process YARN / Mesos / Spark Standalone cluster Tasks sent to executors for processing data Executor Executor Executor Driver launches executors in cluster
  • 11. Spark Streaming Application: Receive data 11 Executor Executor Driver runs receivers as long running tasks Receiver Data stream Driver   object  WordCount  {      def  main(args:  Array[String])  {          val  context  =  new  StreamingContext(...)          val  lines  =  KafkaUtils.createStream(...)          val  words  =  lines.flatMap(_.split("  "))          val  wordCounts  =  words.map(x  =>  (x,1))                                                      .reduceByKey(_  +  _)          wordCounts.print()          context.start()          context.awaitTermination()      }   }   Receiver divides stream into blocks and keeps in memory Data Blocks   Blocks also replicated to another executor Data Blocks  
  • 12. Spark Streaming Application: Process data 12 Executor Executor Receiver Data Blocks   Data Blocks   Data store Every batch interval, driver launches tasks to process the blocks Driver   object  WordCount  {      def  main(args:  Array[String])  {          val  context  =  new  StreamingContext(...)          val  lines  =  KafkaUtils.createStream(...)          val  words  =  lines.flatMap(_.split("  "))          val  wordCounts  =  words.map(x  =>  (x,1))                                                      .reduceByKey(_  +  _)          wordCounts.print()          context.start()          context.awaitTermination()      }   }  
  • 13. Fault-tolerance and Semantics Performance and Stability Monitoring and upgrading 13
  • 14. Failures? Why care? Many streaming applications need zero data loss guarantees despite any kind of failures in the system At least once guarantee – every record processed at least once Exactly once guarantee – every record processed exactly once Different kinds of failures – executor and driver Some failures and guarantee requirements need additional configurations and setups 14
  • 15. Executor Receiver Data Blocks   What if an executor fails? Tasks and receivers restarted by Spark automatically, no config needed 15 Executor Failed Ex. Receiver Blocks   Blocks   Driver If executor fails, receiver is lost and all blocks are lost Receiver Receiver restarted Tasks restarted on block replicas
  • 16. What if the driver fails? 16 Executor Blocks  How do we recover? When the driver fails, all the executors fail All computation, all received blocks are lost Executor Receiver Blocks   Failed Ex. Receiver Blocks   Failed Executor Blocks   Driver Failed Driver
  • 17. Recovering Driver with Checkpointing DStream Checkpointing: Periodically save the DAG of DStreams to fault-tolerant storage 17 Executor Blocks   Executor Receiver Blocks   Active Driver Checkpoint info to HDFS / S3
  • 18. Recovering Driver w/ DStream Checkpointing 18 Failed driver can be restarted from checkpoint information Failed Driver Restarted Driver New Executor New Executor Receiver New executors launched and receivers restarted DStream Checkpointing: Periodically save the DAG of DStreams to fault-tolerant storage
  • 19. Recovering Driver w/ DStream Checkpointing 1.  Configure automatic driver restart All cluster managers support this 2.  Set a checkpoint directory in a HDFS-compatible file system  streamingContext.checkpoint(hdfsDirectory)   3.  Slightly restructure of the code to use checkpoints for recovery ! 19
  • 20. Configurating Automatic Driver Restart Spark Standalone – Use spark-submit with “cluster” mode and “--supervise” See http://spark.apache.org/docs/latest/spark-standalone.html YARN – Use spark-submit in “cluster” mode See YARN config “yarn.resourcemanager.am.max-attempts” Mesos – Marathon can restart Mesos applications 20
  • 21. Restructuring code for Checkpointing 21 val  context  =  new  StreamingContext(...)   val  lines  =  KafkaUtils.createStream(...)   val  words  =  lines.flatMap(...)   ...   context.start()   Create + Setup Start def  creatingFunc():  StreamingContext  =  {          val  context  =  new  StreamingContext(...)            val  lines  =  KafkaUtils.createStream(...)        val  words  =  lines.flatMap(...)        ...        context.checkpoint(hdfsDir)   }   Put all setup code into a function that returns a new StreamingContext Get context setup from HDFS dir OR create a new one with the function val  context  =   StreamingContext.getOrCreate(      hdfsDir,  creatingFunc)   context.start()  
  • 22. Restructuring code for Checkpointing StreamingContext.getOrCreate(): If HDFS directory has checkpoint info recover context from info else call creatingFunc() to create and setup a new context Restarted process can figure out whether to recover using checkpoint info or not 22 def  creatingFunc():  StreamingContext  =  {          val  context  =  new  StreamingContext(...)            val  lines  =  KafkaUtils.createStream(...)        val  words  =  lines.flatMap(...)        ...        context.checkpoint(hdfsDir)   }   val  context  =   StreamingContext.getOrCreate(      hdfsDir,  creatingFunc)   context.start()  
  • 23. Received blocks lost on Restart! 23 Failed Driver Restarted Driver New Executor New Ex. Receiver No Blocks   In-memory blocks of buffered data are lost on driver restart
  • 24. Recovering data with Write Ahead Logs Write Ahead Log (WAL): Synchronously save received data to fault-tolerant storage 24 Executor Blocks saved to HDFS Executor Receiver Blocks   Active Driver Data stream
  • 25. Recovering data with Write Ahead Logs 25 Failed Driver Restarted Driver New Executor New Ex. Receiver Blocks   Blocks recovered from Write Ahead Log Write Ahead Log (WAL): Synchronously save received data to fault-tolerant storage
  • 26. Recovering data with Write Ahead Logs 1.  Enable checkpointing, logs written in checkpoint directory 2.  Enabled WAL in SparkConf configuration          sparkConf.set("spark.streaming.receiver.writeAheadLog.enable",  "true")   3.  Receiver should also be reliable Acknowledge source only after data saved to WAL Unacked data will be replayed from source by restarted receiver 4.  Disable in-memory replication (already replicated by HDFS)  Use StorageLevel.MEMORY_AND_DISK_SER for input DStreams 26
  • 27. RDD Checkpointing Stateful stream processing can lead to long RDD lineages Long lineage = bad for fault-tolerance, too much recomputation RDD checkpointing saves RDD data to the fault-tolerant storage to limit lineage and recomputation More: http://spark.apache.org/docs/latest/streaming-programming-guide.html#checkpointing 27
  • 28. Fault-tolerance Semantics 28 Zero data loss = every stage processes each event at least once despite any failure Sources Transforming Sinks Outputting Receiving
  • 29. Fault-tolerance Semantics 29 Sources Transforming Sinks Outputting Receiving Exactly once, as long as received data is not lost At least once, w/ Checkpointing + WAL + Reliable receiversReceiving Outputting Exactly once, if outputs are idempotent or transactional End-to-end semantics: At-least once
  • 30. Fault-tolerance Semantics 30 Exactly once receiving with new Kafka Direct approach Treats Kafka like a replicated log, reads it like a file Does not use receivers No need to create multiple DStreams and union them No need to enable Write Ahead Logs    val  directKafkaStream  =  KafkaUtils.createDirectStream(...)     https://databricks.com/blog/2015/03/30/improvements-to-kafka-integration-of-spark-streaming.html http://spark.apache.org/docs/latest/streaming-kafka-integration.html Sources Transforming Sinks Outputting Receiving
  • 31. Fault-tolerance Semantics 31 Exactly once receiving with new Kafka Direct approach Sources Transforming Sinks Outputting Receiving Exactly once, as long as received data is not lost Exactly once, if outputs are idempotent or transactional End-to-end semantics: Exactly once!
  • 33. Fault-tolerance and Semantics Performance and Stability Monitoring and Upgrading 33
  • 34. Achieving High Throughput 34 High throughput achieved by sufficient parallelism at all stages of the pipeline Sources Transforming Sinks Outputting Receiving
  • 35. Scaling the Receivers 35 Sources Transforming Sinks Outputting Receiving Sources must be configured with parallel data streams #partitions in Kafka topics, #shards in Kinesis streams, … Streaming app should have multiple receivers that receive the data streams in parallel Multiple input DStreams, each running a receiver Can be unioned together to create one DStream  val  kafkaStream1  =  KafkaUtils.createStream(...)    val  kafkaStream2  =  KafkaUtils.createStream(...)    val  unionedStream  =  kafkaStream1.union(kafkaStream2)    
  • 36. Scaling the Receivers 36 Sources Transforming Sinks Outputting Receiving Sufficient number of executors to run all the receivers Absolute necessity: #cores > #receivers Good rule of thumb: #executors > #receivers, so that no more than 1 receiver per executor, and network is not shared between receivers Kafka Direct approach does not use receivers Automatically parallelizes data reading across executors Parallelism = # Kafka partitions
  • 37. Stability in Processing 37 Sources Transforming Sinks Outputting Receiving For stability, must process data as fast as it is received Must ensure avg batch processing times < batch interval Previous batch is done by the time next batch is received Otherwise, new batches keeps queueing up waiting for previous batches to finish, scheduling delay goes up
  • 38. Reducing Batch Processing Times 38 Sources Transforming Sinks Outputting Receiving More receivers! Executor running receivers do lot of the processing Repartition the received data to explicitly distribute load unionedStream.repartition(40)   Set #partitions in shuffles, make sure its large enough transformedStream.reduceByKey(reduceFunc,  40)   Get more executors and cores!
  • 39. Reducing Batch Processing Times 39 Sources Transforming Sinks Outputting Receiving Use Kryo serialization to serialization costs Register classes for best performance See configurations spark.kryo.* http://spark.apache.org/docs/latest/configuration.html#compression-and-serialization Larger batch durations improve stability More data aggregated together, amortized cost of shuffle Limit ingestion rate to handle data surges See configurations spark.streaming.*maxRate* http://spark.apache.org/docs/latest/configuration.html#spark-streaming
  • 40. Speeding up Output Operations 40 Sources Transforming Sinks Outputting Receiving Write to data stores efficiently dataRDD.foreach  {  event  =>      //  open  connection      //  insert  single  event      //  close  connection   }   foreach: inefficient dataRDD.foreachPartition  {  partition  =>      //  open  connection      //  insert  all  events  in  partition      //  close  connection   }   foreachPartition: efficient dataRDD.foreachPartition  {  partition  =>      //  initialize  pool  or  get  open  connection  from  pool  in  executor      //  insert  all  events  in  partition      //  return  connection  to  pool   }   foreachPartition + connection pool: more efficient
  • 41. Fault-tolerance and Semantics Performance and Stability Monitoring and Upgrading 41
  • 42. Streaming in Spark Web UI Stats over last 1000 batches New in Spark 1.4 42 For stability Scheduling delay should be approx 0 Processing Time approx < batch interval
  • 43. Streaming in Spark Web UI Details of individual batches 43 Details of Spark jobs run in a batch
  • 44. Operational Monitoring Streaming app stats published through Codahale metrics Ganglia sink, Graphite sink, custom Codahale metrics sinks Can see long term trends, across hours and days Configure the metrics using $SPARK_HOME/conf/metrics.properties   Need to compile Spark with Ganglia LGPL profile for Ganglia support (see http://spark.apache.org/docs/latest/monitoring.html#metrics) 44
  • 45. Programmatic Monitoring StreamingListener – Developer interface to get internal events onBatchSubmitted, onBatchStarted, onBatchCompleted, onReceiverStarted, onReceiverStopped, onReceiverError Take a look at StreamingJobProgressListener (private class) for inspiration 45
  • 46. Upgrading Apps 1.  Shutdown your current streaming app gracefully Will process all data before shutting down cleanly streamingContext.stop(stopGracefully  =  true)   2.  Update app code and start it again Cannot upgrade from previous checkpoints if code changes or Spark version changes   46
  • 47. Much to say I have ... but time I have not Memory and GC tuning Using SQLContext DStream.transform operation … Refer to online guide http://spark.apache.org/docs/latest/streaming-programming-guide.html 47
  • 48. Thank you May the stream be with you