SlideShare une entreprise Scribd logo
1  sur  46
Deep dive into Spark Streaming
Tathagata Das (TD)
Matei Zaharia, Haoyuan Li, Timothy Hunter,
Patrick Wendell and many others
UC BERKELEY
What is Spark Streaming?
 Extends Spark for doing large scale stream processing
 Scales to 100s of nodes and achieves second scale latencies
 Efficient and fault-tolerant stateful stream processing
 Integrates with Spark’s batch and interactive processing
 Provides a simple batch-like API for implementing complex
algorithms
Discretized Stream Processing
Run a streaming computation as a series of very small,
deterministic batch jobs
3
Spark
Spark
Streaming
batches of X
seconds
live data stream
processed
results
 Chop up the live stream into batches of X
seconds
 Spark treats each batch of data as RDDs
and processes them using RDD operations
 Finally, the processed results of the RDD
operations are returned in batches
Discretized Stream Processing
Run a streaming computation as a series of very small,
deterministic batch jobs
4
 Batch sizes as low as ½ second, latency ~ 1
second
 Potential for combining batch processing
and streaming processing in the same
system
Spark
Spark
Streaming
batches of X
seconds
live data stream
processed
results
Example 1 – Get hashtags from Twitter
val tweets = ssc.twitterStream(<Twitter username>, <Twitter
password>)
DStream: a sequence of distributed datasets (RDDs)
representing a distributed stream of data
batch @ t+1batch @ t batch @ t+2
tweets DStream
stored in memory as an RDD
(immutable, distributed dataset)
Twitter Streaming API
Example 1 – Get hashtags from Twitter
val tweets = ssc.twitterStream(<Twitter username>, <Twitter
password>)
val hashTags = tweets.flatMap (status => getTags(status))
flatMap flatMap flatMap
…
transformation: modify data in one DStream to create
another DStream
new DStream
new RDDs created
for every batch
batch @ t+1batch @ t batch @ t+2
tweets DStream
hashTags Dstream
*#cat, #dog, … +
Example 1 – Get hashtags from Twitter
val tweets = ssc.twitterStream(<Twitter username>, <Twitter
password>)
val hashTags = tweets.flatMap (status => getTags(status))
hashTags.saveAsHadoopFiles("hdfs://...")
output operation: to push data to external storage
flatMap flatMap flatMap
save save save
batch @ t+1batch @ t batch @ t+2
tweets DStream
hashTags DStream
every batch
saved to HDFS
DStream of data
Example 2 – Count the hashtags over last 1 min
val tweets = ssc.twitterStream(<Twitter username>, <Twitter password>)
val hashTags = tweets.flatMap (status => getTags(status))
val tagCounts = hashTags.window(Minutes(1), Seconds(1)).countByValue()
sliding window
operation
window length sliding interval
window length
sliding interval
tagCounts
Example 2 – Count the hashtags over last 1 min
val tagCounts = hashTags.window(Minutes(1), Seconds(1)).countByValue()
hashTags
t-1 t t+1 t+2 t+3
sliding window
countByValue
count over all
the data in the
window
Key concepts
 DStream – sequence of RDDs representing a stream of data
- Twitter, HDFS, Kafka, Flume, ZeroMQ, Akka Actor, TCP sockets
 Transformations – modify data from one DStream to another
- Standard RDD operations – map, countByValue, reduceByKey, join, …
- Stateful operations – window, countByValueAndWindow, …
 Output Operations – send data to external entity
- saveAsHadoopFiles – saves to HDFS
- foreach – do anything with each batch of results
Arbitrary Stateful Computations
 Maintain arbitrary state, track sessions
- Maintain per-user mood as state, and update it with his/her tweets
moods = tweets.updateStateByKey(tweet => updateMood(tweet))
updateMood(newTweets, lastMood) => newMood
tweets
t-1 t t+1 t+2 t+3
moods
Combine Batch and Stream Processing
 Do arbitrary Spark RDD computation within DStream
- Join incoming tweets with a spam file to filter out bad tweets
tweets.transform(tweetsRDD => {
tweetsRDD.join(spamHDFSFile).filter(...)
})
Fault-tolerance
 RDDs remember the operations
that created them
 Batches of input data are
replicated in memory for fault-
tolerance
 Data lost due to worker failure,
can be recomputed from
replicated input data
 Therefore, all transformed data is
fault-tolerant
input data
replicated
in memory
flatMap
lost partitions
recomputed on
other workers
tweets
RDD
hashTags
RDD
Agenda
 Overview
 DStream Abstraction
 System Model
 Persistence / Caching
 RDD Checkpointing
 Performance Tuning
Discretized Stream (DStream)
A sequence of RDDs representing
a stream of data
What does it take to define a DStream?
DStream Interface
The DStream interface primarily defines how to generate an
RDD in each batch interval
 List of dependent (parent) DStreams
 Slide Interval, the interval at which it will compute RDDs
 Function to compute RDD at a time t
Example: Mapped DStream
 Dependencies: Single parent DStream
 Slide Interval: Same as the parent DStream
 Compute function for time t: Create new RDD by applying map
function on parent DStream’s RDD of time t
Window operation gather together data over a sliding window
 Dependencies: Single parent DStream
 Slide Interval: Window sliding interval
 Compute function for time t: Apply union over all the RDDs of
parent DStream between times t and (t – window length)
Example: Windowed DStream
Parent DStream
window length
sliding interval
Example: Network Input DStream
Base class of all input DStreams that receive data from the network
 Dependencies: None
 Slide Interval: Batch duration in streaming context
 Compute function for time t: Create a BlockRDD with all the
blocks of data received in the last batch interval
 Associated with a Network Receiver object
Network Receiver
Responsible for receiving data and pushing it into Spark’s data
management layer (Block Manager)
Base class for all receivers - Kafka, Flume, etc.
Simple Interface:
 What to do on starting the receiver
- Helper object blockGenerator to push data into Spark
 What to do on stopping the receiver
Example: Socket Receiver
 On start:
Connect to remote TCP server
While socket is connected,
Receiving bytes and deserialize
Deserialize them into Java objects
Add the objects to blockGenerator
 On stop:
Disconnect socket
DStream Graph
t = ssc.twitterStream(“…”)
.map(…)
t.foreach(…)
t1 = ssc.twitterStream(“…”)
t2 = ssc.twitterStream(“…”)
t = t1.union(t2).map(…)
t.saveAsHadoopFiles(…)
t.map(…).foreach(…)
t.filter(…).foreach(…)
T
M
F
E
Twitter Input DStream
Mapped DStream
Foreach DStream
T
U
M
T
M FF
E
F
E
F
E
DStream GraphSpark Streaming program
Dummy DStream signifying
an output operation
DStream Graph  RDD Graphs  Spark jobs
 Every interval, RDD graph is computed from DStream graph
 For each output operation, a Spark action is created
 For each action, a Spark job is created to compute it
T
U
M
T
M FF
E
F
E
F
E
DStream Graph
B
U
M
B
M FA
A A
RDD Graph
Spark actions
Block RDDs with
data received in
last batch interval
3 Spark jobs
Agenda
 Overview
 DStream Abstraction
 System Model
 Persistence / Caching
 RDD Checkpointing
 Performance Tuning
Components
 Network Input Tracker – Keeps track of the data received by each network
receiver and maps them to the corresponding input DStreams
 Job Scheduler – Periodically queries the DStream graph to generate Spark
jobs from received data, and hands them to Job Manager for execution
 Job Manager – Maintains a job queue and executes the jobs in Spark
ssc = new StreamingContext
t =
ssc.twitterStream(“…”)
t.filter(…).foreach(…)
Your program
DStream graph
Spark Context
Network Input Tracker
Job Manager
Job Scheduler
Spark Client
RDD graph Scheduler
Block manager
Shuffle
tracker
Spark Worker
Block
manager
Task
threads
Cluster
Manager
Execution Model – Receiving Data
Spark Streaming + Spark Driver Spark Workers
StreamingContext.start()
Network
Input
Tracker
Receiver
Data recvd
Block
Manager
Blocks replicated
Block
Manager
Master
Block
Manager
Blocks pushed
Spark Workers
Execution Model – Job Scheduling
Network
Input
Tracker
Job
Scheduler Spark’s
Schedulers
Receiver
Block
Manager
Block
Manager
Jobs executed on
worker nodes
DStream
Graph
Job
Manager
JobQueue
Spark Streaming + Spark Driver
Jobs
Block IDsRDDs
Job Scheduling
 Each output operation used generates a job
- More jobs  more time taken to process batches  higher batch
duration
 Job Manager decides how many concurrent Spark jobs to run
- Default is 1, can be set using Java property
spark.streaming.concurrentJobs
- If you have multiple output operations, you can try increasing this
property to reduce batch processing times and so reduce batch duration
Agenda
 Overview
 DStream Abstraction
 System Model
 Persistence / Caching
 RDD Checkpointing
 Performance Tuning
DStream Persistence
 If a DStream is set to persist at a storage level, then all RDDs
generated by it set to the same storage level
 When to persist?
- If there are multiple transformations / actions on a DStream
- If RDDs in a DStream is going to be used multiple times
 Window-based DStreams are automatically persisted in memory
DStream Persistence
 Default storage level of DStreams is StorageLevel.MEMORY_ONLY_SER
(i.e. in memory as serialized bytes)
- Except for input DStreams which have StorageLevel.MEMORY_AND_DISK_SER_2
- Note the difference from RDD’s default level (no serialization)
- Serialization reduces random pauses due to GC providing more consistent
job processing times
Agenda
 Overview
 DStream Abstraction
 System Model
 Persistence / Caching
 RDD Checkpointing
 Performance Tuning
What is RDD checkpointing?
Saving RDD to HDFS to prevent RDD graph from growing too large
 Done internally in Spark transparent to the user program
 Done lazily, saved to HDFS the first time it is computed
red_rdd.checkpoint()
HDFS file
Contents of red_rdd saved
to a HDFS file transparent to
all child RDDs
Why is RDD checkpointing necessary?
Stateful DStream operators can have infinite lineages
Large lineages lead to …
 Large closure of the RDD object  large task sizes  high task launch times
 High recovery times under failure
data
t-1 t t+1 t+2 t+3
states
Why is RDD checkpointing necessary?
Stateful DStream operators can have infinite lineages
Periodic RDD checkpointing solves this
Useful for iterative Spark programs as well
data
t-1 t t+1 t+2 t+3
states
HDF
S
HDF
S
RDD Checkpointing
 Periodicity of checkpoint determines a tradeoff
- Checkpoint too frequent: HDFS writing will slow things down
- Checkpoint too infrequent: Task launch times may increase
- Default setting checkpoints at most once in 10 seconds
- Try to checkpoint once in about 10 batches
Agenda
 Overview
 DStream Abstraction
 System Model
 Persistence / Caching
 RDD Checkpointing
 Performance Tuning
Performance Tuning
Step 1
Achieve a stable configuration that can sustain the
streaming workload
Step 2
Optimize for lower latency
Step 1: Achieving Stable Configuration
How to identify whether a configuration is stable?
 Look for the following messages in the log
Total delay: 0.01500 s for job 12 of time 1371512674000 …
 If the total delay is continuously increasing, then unstable as the
system is unable to process data as fast as its receiving!
 If the total delay stays roughly constant and around 2x the
configured batch duration, then stable
Step 1: Achieving Stable Configuration
How to figure out a good stable configuration?
 Start with a low data rate, small number of nodes, reasonably
large batch duration (5 – 10 seconds)
 Increase the data rate, number of nodes, etc.
 Find the bottleneck in the job processing
- Jobs are divided into stages
- Find which stage is taking the most amount of time
Step 1: Achieving Stable Configuration
How to figure out a good stable configuration?
 If the first map stage on raw data is taking most time, then try …
- Enabling delayed scheduling by setting property spark.locality.wait
- Splitting your data source into multiple sub streams
- Repartitioning the raw data into many partitions as first step
 If any of the subsequent stages are taking a lot of time, try…
- Try increasing the level of parallelism (i.e., increase number of reducers)
- Add more processors to the system
Step 2: Optimize for Lower Latency
 Reduce batch size and find a stable configuration again
- Increase levels of parallelism, etc.
 Optimize serialization overheads
- Consider using Kryo serialization instead of the default Java serialization for
both data and tasks
- For data, set property spark.serializer=spark.KryoSerializer
- For tasks, set spark.closure.serializer=spark.KryoSerializer
 Use Spark stand-alone mode rather than Mesos
Step 2: Optimize for Lower Latency
 Using concurrent mark sweep GC -XX:+UseConcMarkSweepGC is
recommended
- Reduces throughput a little, but also reduces large GC pauses and may
allow lower batch sizes by making processing time more consistent
 Try disabling serialization in DStream/RDD persistence levels
- Increases memory consumption and randomness of GC related pauses, but
may reduce latency by further reducing serialization overheads
 For a full list of guidelines for performance tuning
- Spark Tuning Guide
- Spark Streaming Tuning Guide
Small code base
 5000 LOC for Scala API (+ 1500 LC for Java API)
- Most DStream code mirrors the RDD code
 Easy to take a look and contribute
Future (Possible) Directions
 Better master fault-tolerance
 Better performance for complex queries
- Better performance for stateful processing is a low hanging fruit
 Dashboard for Spark Streaming
- Continuous graphs of processing times, end-to-end latencies
- Drill down for analyzing processing times of stages for finding bottlenecks
 Python API for Spark Streaming
CONTRIBUTIONS ARE WELCOME

Contenu connexe

Tendances

Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark Summit
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Flink Forward
 
A Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLA Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLDatabricks
 
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 -- HasGeekVenkata Naga Ravi
 
Rocks db state store in structured streaming
Rocks db state store in structured streamingRocks db state store in structured streaming
Rocks db state store in structured streamingBalaji Mohanam
 
Temporal-Joins in Kafka Streams and ksqlDB | Matthias Sax, Confluent
Temporal-Joins in Kafka Streams and ksqlDB | Matthias Sax, ConfluentTemporal-Joins in Kafka Streams and ksqlDB | Matthias Sax, Confluent
Temporal-Joins in Kafka Streams and ksqlDB | Matthias Sax, ConfluentHostedbyConfluent
 
Apache Druid Auto Scale-out/in for Streaming Data Ingestion on Kubernetes
Apache Druid Auto Scale-out/in for Streaming Data Ingestion on KubernetesApache Druid Auto Scale-out/in for Streaming Data Ingestion on Kubernetes
Apache Druid Auto Scale-out/in for Streaming Data Ingestion on KubernetesDataWorks Summit
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiDatabricks
 
Whoops, The Numbers Are Wrong! Scaling Data Quality @ Netflix
Whoops, The Numbers Are Wrong! Scaling Data Quality @ NetflixWhoops, The Numbers Are Wrong! Scaling Data Quality @ Netflix
Whoops, The Numbers Are Wrong! Scaling Data Quality @ NetflixDataWorks Summit
 
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Databricks
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internalsKostas Tzoumas
 
Apache Spark overview
Apache Spark overviewApache Spark overview
Apache Spark overviewDataArt
 
Apache Spark Streaming: Architecture and Fault Tolerance
Apache Spark Streaming: Architecture and Fault ToleranceApache Spark Streaming: Architecture and Fault Tolerance
Apache Spark Streaming: Architecture and Fault ToleranceSachin Aggarwal
 
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/AvroThe Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/AvroDatabricks
 
Processing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial ServicesProcessing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial ServicesFlink Forward
 
Best Practice of Compression/Decompression Codes in Apache Spark with Sophia...
 Best Practice of Compression/Decompression Codes in Apache Spark with Sophia... Best Practice of Compression/Decompression Codes in Apache Spark with Sophia...
Best Practice of Compression/Decompression Codes in Apache Spark with Sophia...Databricks
 

Tendances (20)

Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...
 
A Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQLA Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQL
 
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
 
Apache Spark Components
Apache Spark ComponentsApache Spark Components
Apache Spark Components
 
Flink vs. Spark
Flink vs. SparkFlink vs. Spark
Flink vs. Spark
 
Rocks db state store in structured streaming
Rocks db state store in structured streamingRocks db state store in structured streaming
Rocks db state store in structured streaming
 
Temporal-Joins in Kafka Streams and ksqlDB | Matthias Sax, Confluent
Temporal-Joins in Kafka Streams and ksqlDB | Matthias Sax, ConfluentTemporal-Joins in Kafka Streams and ksqlDB | Matthias Sax, Confluent
Temporal-Joins in Kafka Streams and ksqlDB | Matthias Sax, Confluent
 
Apache Druid Auto Scale-out/in for Streaming Data Ingestion on Kubernetes
Apache Druid Auto Scale-out/in for Streaming Data Ingestion on KubernetesApache Druid Auto Scale-out/in for Streaming Data Ingestion on Kubernetes
Apache Druid Auto Scale-out/in for Streaming Data Ingestion on Kubernetes
 
Apache Spark Architecture
Apache Spark ArchitectureApache Spark Architecture
Apache Spark Architecture
 
Spark
SparkSpark
Spark
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and Hudi
 
Whoops, The Numbers Are Wrong! Scaling Data Quality @ Netflix
Whoops, The Numbers Are Wrong! Scaling Data Quality @ NetflixWhoops, The Numbers Are Wrong! Scaling Data Quality @ Netflix
Whoops, The Numbers Are Wrong! Scaling Data Quality @ Netflix
 
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internals
 
Apache Spark overview
Apache Spark overviewApache Spark overview
Apache Spark overview
 
Apache Spark Streaming: Architecture and Fault Tolerance
Apache Spark Streaming: Architecture and Fault ToleranceApache Spark Streaming: Architecture and Fault Tolerance
Apache Spark Streaming: Architecture and Fault Tolerance
 
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/AvroThe Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
 
Processing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial ServicesProcessing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial Services
 
Best Practice of Compression/Decompression Codes in Apache Spark with Sophia...
 Best Practice of Compression/Decompression Codes in Apache Spark with Sophia... Best Practice of Compression/Decompression Codes in Apache Spark with Sophia...
Best Practice of Compression/Decompression Codes in Apache Spark with Sophia...
 

Similaire à Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17

Deep dive into spark streaming
Deep dive into spark streamingDeep dive into spark streaming
Deep dive into spark streamingTao Li
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.pptrveiga100
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.pptAbhijitManna19
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.pptsnowflakebatch
 
strata spark streaming strata spark streamingsrata spark streaming
strata spark streaming strata spark streamingsrata spark streamingstrata spark streaming strata spark streamingsrata spark streaming
strata spark streaming strata spark streamingsrata spark streamingShidrokhGoudarzi1
 
Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...
Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...
Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...Tathagata Das
 
Meet Up - Spark Stream Processing + Kafka
Meet Up - Spark Stream Processing + KafkaMeet Up - Spark Stream Processing + Kafka
Meet Up - Spark Stream Processing + KafkaKnoldus Inc.
 
Spark Streaming with Cassandra
Spark Streaming with CassandraSpark Streaming with Cassandra
Spark Streaming with CassandraJacek Lewandowski
 
Spark 计算模型
Spark 计算模型Spark 计算模型
Spark 计算模型wang xing
 
Unified Big Data Processing with Apache Spark (QCON 2014)
Unified Big Data Processing with Apache Spark (QCON 2014)Unified Big Data Processing with Apache Spark (QCON 2014)
Unified Big Data Processing with Apache Spark (QCON 2014)Databricks
 
Unified Big Data Processing with Apache Spark
Unified Big Data Processing with Apache SparkUnified Big Data Processing with Apache Spark
Unified Big Data Processing with Apache SparkC4Media
 
No more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in productionNo more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in productionChetan Khatri
 
Productionizing your Streaming Jobs
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming JobsDatabricks
 
Introduction to Spark Streaming
Introduction to Spark StreamingIntroduction to Spark Streaming
Introduction to Spark StreamingKnoldus Inc.
 
Spark & Spark Streaming Internals - Nov 15 (1)
Spark & Spark Streaming Internals - Nov 15 (1)Spark & Spark Streaming Internals - Nov 15 (1)
Spark & Spark Streaming Internals - Nov 15 (1)Akhil Das
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkDatabricks
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Guido Schmutz
 

Similaire à Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17 (20)

Deep dive into spark streaming
Deep dive into spark streamingDeep dive into spark streaming
Deep dive into spark streaming
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.ppt
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.ppt
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.ppt
 
strata spark streaming strata spark streamingsrata spark streaming
strata spark streaming strata spark streamingsrata spark streamingstrata spark streaming strata spark streamingsrata spark streaming
strata spark streaming strata spark streamingsrata spark streaming
 
Spark streaming
Spark streamingSpark streaming
Spark streaming
 
Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...
Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...
Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms ...
 
So you think you can stream.pptx
So you think you can stream.pptxSo you think you can stream.pptx
So you think you can stream.pptx
 
Spark streaming
Spark streamingSpark streaming
Spark streaming
 
Meet Up - Spark Stream Processing + Kafka
Meet Up - Spark Stream Processing + KafkaMeet Up - Spark Stream Processing + Kafka
Meet Up - Spark Stream Processing + Kafka
 
Spark Streaming with Cassandra
Spark Streaming with CassandraSpark Streaming with Cassandra
Spark Streaming with Cassandra
 
Spark 计算模型
Spark 计算模型Spark 计算模型
Spark 计算模型
 
Unified Big Data Processing with Apache Spark (QCON 2014)
Unified Big Data Processing with Apache Spark (QCON 2014)Unified Big Data Processing with Apache Spark (QCON 2014)
Unified Big Data Processing with Apache Spark (QCON 2014)
 
Unified Big Data Processing with Apache Spark
Unified Big Data Processing with Apache SparkUnified Big Data Processing with Apache Spark
Unified Big Data Processing with Apache Spark
 
No more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in productionNo more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in production
 
Productionizing your Streaming Jobs
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming Jobs
 
Introduction to Spark Streaming
Introduction to Spark StreamingIntroduction to Spark Streaming
Introduction to Spark Streaming
 
Spark & Spark Streaming Internals - Nov 15 (1)
Spark & Spark Streaming Internals - Nov 15 (1)Spark & Spark Streaming Internals - Nov 15 (1)
Spark & Spark Streaming Internals - Nov 15 (1)
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
 

Dernier

New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024BookNet Canada
 
Advanced Computer Architecture – An Introduction
Advanced Computer Architecture – An IntroductionAdvanced Computer Architecture – An Introduction
Advanced Computer Architecture – An IntroductionDilum Bandara
 
The State of Passkeys with FIDO Alliance.pptx
The State of Passkeys with FIDO Alliance.pptxThe State of Passkeys with FIDO Alliance.pptx
The State of Passkeys with FIDO Alliance.pptxLoriGlavin3
 
A Deep Dive on Passkeys: FIDO Paris Seminar.pptx
A Deep Dive on Passkeys: FIDO Paris Seminar.pptxA Deep Dive on Passkeys: FIDO Paris Seminar.pptx
A Deep Dive on Passkeys: FIDO Paris Seminar.pptxLoriGlavin3
 
Dev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebDev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebUiPathCommunity
 
What is DBT - The Ultimate Data Build Tool.pdf
What is DBT - The Ultimate Data Build Tool.pdfWhat is DBT - The Ultimate Data Build Tool.pdf
What is DBT - The Ultimate Data Build Tool.pdfMounikaPolabathina
 
"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii SoldatenkoFwdays
 
Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 365Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 3652toLead Limited
 
Developer Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLDeveloper Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLScyllaDB
 
SALESFORCE EDUCATION CLOUD | FEXLE SERVICES
SALESFORCE EDUCATION CLOUD | FEXLE SERVICESSALESFORCE EDUCATION CLOUD | FEXLE SERVICES
SALESFORCE EDUCATION CLOUD | FEXLE SERVICESmohitsingh558521
 
TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024Lonnie McRorey
 
What's New in Teams Calling, Meetings and Devices March 2024
What's New in Teams Calling, Meetings and Devices March 2024What's New in Teams Calling, Meetings and Devices March 2024
What's New in Teams Calling, Meetings and Devices March 2024Stephanie Beckett
 
Moving Beyond Passwords: FIDO Paris Seminar.pdf
Moving Beyond Passwords: FIDO Paris Seminar.pdfMoving Beyond Passwords: FIDO Paris Seminar.pdf
Moving Beyond Passwords: FIDO Paris Seminar.pdfLoriGlavin3
 
Sample pptx for embedding into website for demo
Sample pptx for embedding into website for demoSample pptx for embedding into website for demo
Sample pptx for embedding into website for demoHarshalMandlekar2
 
DevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsDevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsSergiu Bodiu
 
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxThe Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxLoriGlavin3
 
Artificial intelligence in cctv survelliance.pptx
Artificial intelligence in cctv survelliance.pptxArtificial intelligence in cctv survelliance.pptx
Artificial intelligence in cctv survelliance.pptxhariprasad279825
 
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxDigital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxLoriGlavin3
 
A Journey Into the Emotions of Software Developers
A Journey Into the Emotions of Software DevelopersA Journey Into the Emotions of Software Developers
A Journey Into the Emotions of Software DevelopersNicole Novielli
 
Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!Commit University
 

Dernier (20)

New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
 
Advanced Computer Architecture – An Introduction
Advanced Computer Architecture – An IntroductionAdvanced Computer Architecture – An Introduction
Advanced Computer Architecture – An Introduction
 
The State of Passkeys with FIDO Alliance.pptx
The State of Passkeys with FIDO Alliance.pptxThe State of Passkeys with FIDO Alliance.pptx
The State of Passkeys with FIDO Alliance.pptx
 
A Deep Dive on Passkeys: FIDO Paris Seminar.pptx
A Deep Dive on Passkeys: FIDO Paris Seminar.pptxA Deep Dive on Passkeys: FIDO Paris Seminar.pptx
A Deep Dive on Passkeys: FIDO Paris Seminar.pptx
 
Dev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebDev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio Web
 
What is DBT - The Ultimate Data Build Tool.pdf
What is DBT - The Ultimate Data Build Tool.pdfWhat is DBT - The Ultimate Data Build Tool.pdf
What is DBT - The Ultimate Data Build Tool.pdf
 
"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko
 
Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 365Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 365
 
Developer Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLDeveloper Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQL
 
SALESFORCE EDUCATION CLOUD | FEXLE SERVICES
SALESFORCE EDUCATION CLOUD | FEXLE SERVICESSALESFORCE EDUCATION CLOUD | FEXLE SERVICES
SALESFORCE EDUCATION CLOUD | FEXLE SERVICES
 
TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024
 
What's New in Teams Calling, Meetings and Devices March 2024
What's New in Teams Calling, Meetings and Devices March 2024What's New in Teams Calling, Meetings and Devices March 2024
What's New in Teams Calling, Meetings and Devices March 2024
 
Moving Beyond Passwords: FIDO Paris Seminar.pdf
Moving Beyond Passwords: FIDO Paris Seminar.pdfMoving Beyond Passwords: FIDO Paris Seminar.pdf
Moving Beyond Passwords: FIDO Paris Seminar.pdf
 
Sample pptx for embedding into website for demo
Sample pptx for embedding into website for demoSample pptx for embedding into website for demo
Sample pptx for embedding into website for demo
 
DevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsDevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platforms
 
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxThe Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
 
Artificial intelligence in cctv survelliance.pptx
Artificial intelligence in cctv survelliance.pptxArtificial intelligence in cctv survelliance.pptx
Artificial intelligence in cctv survelliance.pptx
 
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxDigital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
 
A Journey Into the Emotions of Software Developers
A Journey Into the Emotions of Software DevelopersA Journey Into the Emotions of Software Developers
A Journey Into the Emotions of Software Developers
 
Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!
 

Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17

  • 1. Deep dive into Spark Streaming Tathagata Das (TD) Matei Zaharia, Haoyuan Li, Timothy Hunter, Patrick Wendell and many others UC BERKELEY
  • 2. What is Spark Streaming?  Extends Spark for doing large scale stream processing  Scales to 100s of nodes and achieves second scale latencies  Efficient and fault-tolerant stateful stream processing  Integrates with Spark’s batch and interactive processing  Provides a simple batch-like API for implementing complex algorithms
  • 3. Discretized Stream Processing Run a streaming computation as a series of very small, deterministic batch jobs 3 Spark Spark Streaming batches of X seconds live data stream processed results  Chop up the live stream into batches of X seconds  Spark treats each batch of data as RDDs and processes them using RDD operations  Finally, the processed results of the RDD operations are returned in batches
  • 4. Discretized Stream Processing Run a streaming computation as a series of very small, deterministic batch jobs 4  Batch sizes as low as ½ second, latency ~ 1 second  Potential for combining batch processing and streaming processing in the same system Spark Spark Streaming batches of X seconds live data stream processed results
  • 5. Example 1 – Get hashtags from Twitter val tweets = ssc.twitterStream(<Twitter username>, <Twitter password>) DStream: a sequence of distributed datasets (RDDs) representing a distributed stream of data batch @ t+1batch @ t batch @ t+2 tweets DStream stored in memory as an RDD (immutable, distributed dataset) Twitter Streaming API
  • 6. Example 1 – Get hashtags from Twitter val tweets = ssc.twitterStream(<Twitter username>, <Twitter password>) val hashTags = tweets.flatMap (status => getTags(status)) flatMap flatMap flatMap … transformation: modify data in one DStream to create another DStream new DStream new RDDs created for every batch batch @ t+1batch @ t batch @ t+2 tweets DStream hashTags Dstream *#cat, #dog, … +
  • 7. Example 1 – Get hashtags from Twitter val tweets = ssc.twitterStream(<Twitter username>, <Twitter password>) val hashTags = tweets.flatMap (status => getTags(status)) hashTags.saveAsHadoopFiles("hdfs://...") output operation: to push data to external storage flatMap flatMap flatMap save save save batch @ t+1batch @ t batch @ t+2 tweets DStream hashTags DStream every batch saved to HDFS
  • 8. DStream of data Example 2 – Count the hashtags over last 1 min val tweets = ssc.twitterStream(<Twitter username>, <Twitter password>) val hashTags = tweets.flatMap (status => getTags(status)) val tagCounts = hashTags.window(Minutes(1), Seconds(1)).countByValue() sliding window operation window length sliding interval window length sliding interval
  • 9. tagCounts Example 2 – Count the hashtags over last 1 min val tagCounts = hashTags.window(Minutes(1), Seconds(1)).countByValue() hashTags t-1 t t+1 t+2 t+3 sliding window countByValue count over all the data in the window
  • 10. Key concepts  DStream – sequence of RDDs representing a stream of data - Twitter, HDFS, Kafka, Flume, ZeroMQ, Akka Actor, TCP sockets  Transformations – modify data from one DStream to another - Standard RDD operations – map, countByValue, reduceByKey, join, … - Stateful operations – window, countByValueAndWindow, …  Output Operations – send data to external entity - saveAsHadoopFiles – saves to HDFS - foreach – do anything with each batch of results
  • 11. Arbitrary Stateful Computations  Maintain arbitrary state, track sessions - Maintain per-user mood as state, and update it with his/her tweets moods = tweets.updateStateByKey(tweet => updateMood(tweet)) updateMood(newTweets, lastMood) => newMood tweets t-1 t t+1 t+2 t+3 moods
  • 12. Combine Batch and Stream Processing  Do arbitrary Spark RDD computation within DStream - Join incoming tweets with a spam file to filter out bad tweets tweets.transform(tweetsRDD => { tweetsRDD.join(spamHDFSFile).filter(...) })
  • 13. Fault-tolerance  RDDs remember the operations that created them  Batches of input data are replicated in memory for fault- tolerance  Data lost due to worker failure, can be recomputed from replicated input data  Therefore, all transformed data is fault-tolerant input data replicated in memory flatMap lost partitions recomputed on other workers tweets RDD hashTags RDD
  • 14. Agenda  Overview  DStream Abstraction  System Model  Persistence / Caching  RDD Checkpointing  Performance Tuning
  • 15. Discretized Stream (DStream) A sequence of RDDs representing a stream of data What does it take to define a DStream?
  • 16. DStream Interface The DStream interface primarily defines how to generate an RDD in each batch interval  List of dependent (parent) DStreams  Slide Interval, the interval at which it will compute RDDs  Function to compute RDD at a time t
  • 17. Example: Mapped DStream  Dependencies: Single parent DStream  Slide Interval: Same as the parent DStream  Compute function for time t: Create new RDD by applying map function on parent DStream’s RDD of time t
  • 18. Window operation gather together data over a sliding window  Dependencies: Single parent DStream  Slide Interval: Window sliding interval  Compute function for time t: Apply union over all the RDDs of parent DStream between times t and (t – window length) Example: Windowed DStream Parent DStream window length sliding interval
  • 19. Example: Network Input DStream Base class of all input DStreams that receive data from the network  Dependencies: None  Slide Interval: Batch duration in streaming context  Compute function for time t: Create a BlockRDD with all the blocks of data received in the last batch interval  Associated with a Network Receiver object
  • 20. Network Receiver Responsible for receiving data and pushing it into Spark’s data management layer (Block Manager) Base class for all receivers - Kafka, Flume, etc. Simple Interface:  What to do on starting the receiver - Helper object blockGenerator to push data into Spark  What to do on stopping the receiver
  • 21. Example: Socket Receiver  On start: Connect to remote TCP server While socket is connected, Receiving bytes and deserialize Deserialize them into Java objects Add the objects to blockGenerator  On stop: Disconnect socket
  • 22. DStream Graph t = ssc.twitterStream(“…”) .map(…) t.foreach(…) t1 = ssc.twitterStream(“…”) t2 = ssc.twitterStream(“…”) t = t1.union(t2).map(…) t.saveAsHadoopFiles(…) t.map(…).foreach(…) t.filter(…).foreach(…) T M F E Twitter Input DStream Mapped DStream Foreach DStream T U M T M FF E F E F E DStream GraphSpark Streaming program Dummy DStream signifying an output operation
  • 23. DStream Graph  RDD Graphs  Spark jobs  Every interval, RDD graph is computed from DStream graph  For each output operation, a Spark action is created  For each action, a Spark job is created to compute it T U M T M FF E F E F E DStream Graph B U M B M FA A A RDD Graph Spark actions Block RDDs with data received in last batch interval 3 Spark jobs
  • 24. Agenda  Overview  DStream Abstraction  System Model  Persistence / Caching  RDD Checkpointing  Performance Tuning
  • 25. Components  Network Input Tracker – Keeps track of the data received by each network receiver and maps them to the corresponding input DStreams  Job Scheduler – Periodically queries the DStream graph to generate Spark jobs from received data, and hands them to Job Manager for execution  Job Manager – Maintains a job queue and executes the jobs in Spark ssc = new StreamingContext t = ssc.twitterStream(“…”) t.filter(…).foreach(…) Your program DStream graph Spark Context Network Input Tracker Job Manager Job Scheduler Spark Client RDD graph Scheduler Block manager Shuffle tracker Spark Worker Block manager Task threads Cluster Manager
  • 26. Execution Model – Receiving Data Spark Streaming + Spark Driver Spark Workers StreamingContext.start() Network Input Tracker Receiver Data recvd Block Manager Blocks replicated Block Manager Master Block Manager Blocks pushed
  • 27. Spark Workers Execution Model – Job Scheduling Network Input Tracker Job Scheduler Spark’s Schedulers Receiver Block Manager Block Manager Jobs executed on worker nodes DStream Graph Job Manager JobQueue Spark Streaming + Spark Driver Jobs Block IDsRDDs
  • 28. Job Scheduling  Each output operation used generates a job - More jobs  more time taken to process batches  higher batch duration  Job Manager decides how many concurrent Spark jobs to run - Default is 1, can be set using Java property spark.streaming.concurrentJobs - If you have multiple output operations, you can try increasing this property to reduce batch processing times and so reduce batch duration
  • 29. Agenda  Overview  DStream Abstraction  System Model  Persistence / Caching  RDD Checkpointing  Performance Tuning
  • 30. DStream Persistence  If a DStream is set to persist at a storage level, then all RDDs generated by it set to the same storage level  When to persist? - If there are multiple transformations / actions on a DStream - If RDDs in a DStream is going to be used multiple times  Window-based DStreams are automatically persisted in memory
  • 31. DStream Persistence  Default storage level of DStreams is StorageLevel.MEMORY_ONLY_SER (i.e. in memory as serialized bytes) - Except for input DStreams which have StorageLevel.MEMORY_AND_DISK_SER_2 - Note the difference from RDD’s default level (no serialization) - Serialization reduces random pauses due to GC providing more consistent job processing times
  • 32. Agenda  Overview  DStream Abstraction  System Model  Persistence / Caching  RDD Checkpointing  Performance Tuning
  • 33. What is RDD checkpointing? Saving RDD to HDFS to prevent RDD graph from growing too large  Done internally in Spark transparent to the user program  Done lazily, saved to HDFS the first time it is computed red_rdd.checkpoint() HDFS file Contents of red_rdd saved to a HDFS file transparent to all child RDDs
  • 34. Why is RDD checkpointing necessary? Stateful DStream operators can have infinite lineages Large lineages lead to …  Large closure of the RDD object  large task sizes  high task launch times  High recovery times under failure data t-1 t t+1 t+2 t+3 states
  • 35. Why is RDD checkpointing necessary? Stateful DStream operators can have infinite lineages Periodic RDD checkpointing solves this Useful for iterative Spark programs as well data t-1 t t+1 t+2 t+3 states HDF S HDF S
  • 36. RDD Checkpointing  Periodicity of checkpoint determines a tradeoff - Checkpoint too frequent: HDFS writing will slow things down - Checkpoint too infrequent: Task launch times may increase - Default setting checkpoints at most once in 10 seconds - Try to checkpoint once in about 10 batches
  • 37. Agenda  Overview  DStream Abstraction  System Model  Persistence / Caching  RDD Checkpointing  Performance Tuning
  • 38. Performance Tuning Step 1 Achieve a stable configuration that can sustain the streaming workload Step 2 Optimize for lower latency
  • 39. Step 1: Achieving Stable Configuration How to identify whether a configuration is stable?  Look for the following messages in the log Total delay: 0.01500 s for job 12 of time 1371512674000 …  If the total delay is continuously increasing, then unstable as the system is unable to process data as fast as its receiving!  If the total delay stays roughly constant and around 2x the configured batch duration, then stable
  • 40. Step 1: Achieving Stable Configuration How to figure out a good stable configuration?  Start with a low data rate, small number of nodes, reasonably large batch duration (5 – 10 seconds)  Increase the data rate, number of nodes, etc.  Find the bottleneck in the job processing - Jobs are divided into stages - Find which stage is taking the most amount of time
  • 41. Step 1: Achieving Stable Configuration How to figure out a good stable configuration?  If the first map stage on raw data is taking most time, then try … - Enabling delayed scheduling by setting property spark.locality.wait - Splitting your data source into multiple sub streams - Repartitioning the raw data into many partitions as first step  If any of the subsequent stages are taking a lot of time, try… - Try increasing the level of parallelism (i.e., increase number of reducers) - Add more processors to the system
  • 42. Step 2: Optimize for Lower Latency  Reduce batch size and find a stable configuration again - Increase levels of parallelism, etc.  Optimize serialization overheads - Consider using Kryo serialization instead of the default Java serialization for both data and tasks - For data, set property spark.serializer=spark.KryoSerializer - For tasks, set spark.closure.serializer=spark.KryoSerializer  Use Spark stand-alone mode rather than Mesos
  • 43. Step 2: Optimize for Lower Latency  Using concurrent mark sweep GC -XX:+UseConcMarkSweepGC is recommended - Reduces throughput a little, but also reduces large GC pauses and may allow lower batch sizes by making processing time more consistent  Try disabling serialization in DStream/RDD persistence levels - Increases memory consumption and randomness of GC related pauses, but may reduce latency by further reducing serialization overheads  For a full list of guidelines for performance tuning - Spark Tuning Guide - Spark Streaming Tuning Guide
  • 44. Small code base  5000 LOC for Scala API (+ 1500 LC for Java API) - Most DStream code mirrors the RDD code  Easy to take a look and contribute
  • 45. Future (Possible) Directions  Better master fault-tolerance  Better performance for complex queries - Better performance for stateful processing is a low hanging fruit  Dashboard for Spark Streaming - Continuous graphs of processing times, end-to-end latencies - Drill down for analyzing processing times of stages for finding bottlenecks  Python API for Spark Streaming