SlideShare a Scribd company logo
1 of 41
Streaming items through a
cluster with Spark Streaming
Tathagata “TD” Das
@tathadas
CME 323: Distributed Algorithms and Optimization
Stanford, May 6, 2015
Who am I?
> Project Management Committee (PMC)
member of Apache Spark
> Lead developer of Spark Streaming
> Formerly in AMPLab, UC Berkeley
> Software developer at Databricks
> Databricks was started by creators of Spark to
provide Spark-as-a-service in the cloud
Big Data
Big Streaming Data
Why process Big Streaming
Data?
Fraud detection in bank transactions
Anomalies in sensor data
Cat videos in tweets
How to Process Big Streaming
Data
> Ingest – Receive and buffer the streaming data
> Process – Clean, extract, transform the data
> Store – Store transformed data for consumption
Ingest
data
Process
data
Store
results
Raw Tweets
How to Process Big Streaming
Data
> For big streams, every step requires a cluster
> Every step requires a system that is designed for
it
Ingest
data
Process
data
Store
results
Raw Tweets
Stream Ingestion Systems
> Kafka – popular distributed pub-sub system
> Kinesis – Amazon managed distributed pub-sub
> Flume – like a distributed data pipe
Ingest
data
Process
data
Store
results
Raw Tweets
Amazon
Kinesis
Stream Ingestion Systems
> Spark Streaming – From UC Berkeley + Databricks
> Storm – From Backtype + Twitter
> Samza – From LinkedIn
Ingest
data
Process
data
Store
results
Raw Tweets
Stream Ingestion Systems
> File systems – HDFS, Amazon S3, etc.
> Key-value stores – HBase, Cassandra, etc.
> Databases – MongoDB, MemSQL, etc.
Ingest
data
Process
data
Store
results
Raw Tweets
Kafka Cluster
Producers and
Consumers
> Producers publish data tagged by “topic”
> Consumers subscribe to data of a particular “topic”
Producer 1
Producer 2
(topicX, data1)
(topicY, data2)
(topicX, data3)
Topic X
Consumer
Topic Y
Consumer
(topicX, data1)
(topicX, data3)
(topicY, data2)
> Topic = category of message, divided into partitions
> Partition = ordered, numbered stream of messages
> Producer decides which (topic, partition) to put each
message in
Topics and Partitions
> Topic = category of message, divided into
partitions
> Partition = ordered, numbered stream of messages
> Producer decides which (topic, partition) to put
each message in
> Consumer decides which (topic, partition) to pull
messages from
- High-level consumer – handles fault-recovery with
Zookeeper
- Simple consumer – low-level API for greater control
Topics and Partitions
How to process Kafka
messages?
Ingest
data
Process
data
Store
results
Raw Tweets
> Incoming tweets received in distributed manner
and buffered in Kafka
> How to process them?
treaming
What is Spark Streaming?
Scalable, fault-tolerant stream processing system
File systems
Databases
Dashboards
Flume
HDFS
Kinesis
Kafka
Twitter
High-level API
joins, windows, …
often 5x less code
Fault-tolerant
Exactly-once semantics,
even for stateful ops
Integration
Integrate with MLlib, SQL,
DataFrames, GraphX
How does Spark Streaming
work?
> Receivers chop up data streams into batches of
few seconds
> Spark processing engine processes each batch
and pushes out the results to external data
stores
data streams
Receivers
batches
as RDDs
results as
RDDs
Spark Programming Model
> Resilient distributed datasets (RDDs)
- Distributed, partitioned collection of objects
- Manipulated through parallel transformations
(map, filter, reduceByKey, …)
- All transformations are lazy, execution forced by
actions
(count, reduce, take, …)
- Can be cached in memory across cluster
- Automatically rebuilt on failure
Spark Streaming Programming
Model
> Discretized Stream (DStream)
- Represents a stream of data
- Implemented as a infinite sequence of RDDs
> DStreams API very similar to RDD API
- Functional APIs in
- Create input DStreams from Kafka, Flume, Kinesis, HDFS,
…
- Apply transformations
Example – Get hashtags from
Twitter
val ssc = new StreamingContext(conf, Seconds(1))
StreamingContext is the starting
point of all streaming functionality
Batch interval, by which
streams will be chopped up
Example – Get hashtags from
Twitter
val ssc = new StreamingContext(conf, Seconds(1))
val tweets = TwitterUtils.createStream(ssc, auth)
Input DStream
batch @ t+1batch @ t batch @ t+2
tweets DStream
replicated and stored in
memory as RDDs
Twitter Streaming API
Example – Get hashtags from
Twitter
val tweets = TwitterUtils.createStream(ssc, None)
val hashTags = tweets.flatMap(status => getTags(status))
flatMap flatMap flatMap
…
transformation: modify data in one
DStream to create another DStream
transformed
DStream
new RDDs created
for every batch
batch @ t+1batch @ t batch @ t+2
tweets DStream
hashTags Dstream
[#cat, #dog, … ]
Example – Get hashtags from
Twitter
val tweets = TwitterUtils.createStream(ssc, None)
val hashTags = tweets.flatMap(status => getTags(status))
hashTags.saveAsTextFiles("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
Example – Get hashtags from
Twitter
val tweets = TwitterUtils.createStream(ssc, None)
val hashTags = tweets.flatMap(status => getTags(status))
hashTags.foreachRDD(hashTagRDD => { ... })
foreachRDD: do whatever you want with the processed data
flatMap flatMap flatMap
foreach foreach foreach
batch @ t+1batch @ t batch @ t+2
tweets DStream
hashTags DStream
Write to a database, update analytics
UI, do whatever you want
Example – Get hashtags from
Twitter
val tweets = TwitterUtils.createStream(ssc, None)
val hashTags = tweets.flatMap(status => getTags(status))
hashTags.foreachRDD(hashTagRDD => { ... })
ssc.start()
all of this was just setup for what to do when
streaming data is receiver
this actually starts the receiving and processing
What’s going on inside?
> Receiver buffers tweets
in Executors’ memory
> Spark Streaming Driver
launches tasks to
process tweets
Driver
running
DStreams
Executors
launch tasks to
process tweets
Raw Tweets
Twitter
Receiver
Buffered
Tweets
Buffered
Tweets
Spark Cluster
What’s going on inside?
Driver
running
DStreams
Executors
launch tasks to
process data
Receiver
Kafka Cluster
ReceiverReceiver
receive data in
parallel
Spark Cluster
Performance
Can process 60M records/sec (6 GB/sec) on
100 nodes at sub-second latency
0
0.5
1
1.5
2
2.5
3
3.5
0 50 100
ClusterThroughput(GB/s)
# Nodes in Cluster
WordCount
1 sec
2 sec
0
1
2
3
4
5
6
7
0 50 100
ClusterThhroughput(GB/s)
# Nodes in Cluster
Grep
1 sec
2 sec
DStream of data
Window-based Transformations
val tweets = TwitterUtils.createStream(ssc, auth)
val hashTags = tweets.flatMap(status => getTags(status))
val tagCounts = hashTags.window(Minutes(1), Seconds(5)).countByValue()
sliding window
operation
window length sliding interval
window length
sliding interval
Arbitrary Stateful Computations
Specify function to generate new state based on
previous state and new data
- Example: Maintain per-user mood as state, and update
it with their tweets
def updateMood(newTweets, lastMood) => newMood
val moods = tweetsByUser.updateStateByKey(updateMood _)
Integrates with Spark
Ecosystem
Spark Core
Spark
Streaming
Spark
SQL
MLlib GraphX
Combine batch and streaming
processing
> Join data streams with static data sets
// Create data set from Hadoop file
val dataset = sparkContext.hadoopFile(“file”)
// Join each batch in stream with dataset
kafkaStream.transform { batchRDD =>
batchRDD.join(dataset)filter(...)
}
Spark Core
Spark
Streaming
Spark SQL MLlib GraphX
Combine machine learning with
streaming
> Learn models offline, apply them online
// Learn model offline
val model = KMeans.train(dataset, ...)
// Apply model online on stream
kafkaStream.map { event =>
model.predict(event.feature)
}
Spark Core
Spark
Streaming
Spark SQL MLlib GraphX
Combine SQL with streaming
> Interactively query streaming data with SQL
// Register each batch in stream as table
kafkaStream.map { batchRDD =>
batchRDD.registerTempTable("latestEvents")
}
// Interactively query table
sqlContext.sql("select * from latestEvents")
Spark Core
Spark
Streaming
Spark SQL MLlib GraphX
100+ known industry
deployments
Why are they adopting Spark
Streaming?
Easy, high-level API
Unified API across batch and streaming
Integration with Spark SQL and MLlib
Ease of operations
37
Neuroscience @ Freeman Lab, Janelia Farm
Spark Streaming and MLlib
to analyze neural activities
Laser microscope scans
Zebrafish brain Spark
Streaming  interactive
visualization 
laser ZAP to kill neurons!
http://www.jeremyfreeman.net/share/talks/spark-summit-2014/
Neuroscience @ Freeman Lab, Janelia Farm
Streaming machine learning
algorithms on time series
data of every neuron
Upto 2TB/hour and
increasing with brain size
Upto 80 HPC nodes
http://www.jeremyfreeman.net/share/talks/spark-summit-2014/
Streaming Machine Learning
Algos
> Streaming Linear Regression
> Streaming Logistic Regression
> Streaming KMeans
http://www.jeremyfreeman.net/share/talks/spark-summit-east-2015/#/algorithms-repeat
Okay okay, how do I start off?
> Online Streaming
Programming Guide
http://spark.apache.org/docs/latest/streaming-programming-guide.html
> Streaming examples
https://github.com/apache/spark/tree/master/examples/src/main/scala/o
rg/apache/spark/examples/streaming

More Related Content

What's hot

A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...Databricks
 
Spark and spark streaming internals
Spark and spark streaming internalsSpark and spark streaming internals
Spark and spark streaming internalsSigmoid
 
Distributed real time stream processing- why and how
Distributed real time stream processing- why and howDistributed real time stream processing- why and how
Distributed real time stream processing- why and howPetr Zapletal
 
What's new with Apache Spark's Structured Streaming?
What's new with Apache Spark's Structured Streaming?What's new with Apache Spark's Structured Streaming?
What's new with Apache Spark's Structured Streaming?Miklos Christine
 
GoodFit: Multi-Resource Packing of Tasks with Dependencies
GoodFit: Multi-Resource Packing of Tasks with DependenciesGoodFit: Multi-Resource Packing of Tasks with Dependencies
GoodFit: Multi-Resource Packing of Tasks with DependenciesDataWorks Summit/Hadoop Summit
 
Predictive Maintenance at the Dutch Railways with Ivo Everts
Predictive Maintenance at the Dutch Railways with Ivo EvertsPredictive Maintenance at the Dutch Railways with Ivo Everts
Predictive Maintenance at the Dutch Railways with Ivo EvertsDatabricks
 
Hadoop & MapReduce
Hadoop & MapReduceHadoop & MapReduce
Hadoop & MapReduceNewvewm
 
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
 
Large volume data analysis on the Typesafe Reactive Platform
Large volume data analysis on the Typesafe Reactive PlatformLarge volume data analysis on the Typesafe Reactive Platform
Large volume data analysis on the Typesafe Reactive PlatformMartin Zapletal
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkDatabricks
 
Distributed Stream Processing - Spark Summit East 2017
Distributed Stream Processing - Spark Summit East 2017Distributed Stream Processing - Spark Summit East 2017
Distributed Stream Processing - Spark Summit East 2017Petr Zapletal
 
OpenTSDB 2.0
OpenTSDB 2.0OpenTSDB 2.0
OpenTSDB 2.0HBaseCon
 
Scaling Apache Storm - Strata + Hadoop World 2014
Scaling Apache Storm - Strata + Hadoop World 2014Scaling Apache Storm - Strata + Hadoop World 2014
Scaling Apache Storm - Strata + Hadoop World 2014P. Taylor Goetz
 
Introduction to MapReduce and Hadoop
Introduction to MapReduce and HadoopIntroduction to MapReduce and Hadoop
Introduction to MapReduce and HadoopMohamed Elsaka
 
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Databricks
 
Spark Study Notes
Spark Study NotesSpark Study Notes
Spark Study NotesRichard Kuo
 

What's hot (20)

A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
A Deep Dive into Stateful Stream Processing in Structured Streaming with Tath...
 
Apache Spark
Apache Spark Apache Spark
Apache Spark
 
Spark and spark streaming internals
Spark and spark streaming internalsSpark and spark streaming internals
Spark and spark streaming internals
 
Distributed real time stream processing- why and how
Distributed real time stream processing- why and howDistributed real time stream processing- why and how
Distributed real time stream processing- why and how
 
What's new with Apache Spark's Structured Streaming?
What's new with Apache Spark's Structured Streaming?What's new with Apache Spark's Structured Streaming?
What's new with Apache Spark's Structured Streaming?
 
Apache Spark Architecture
Apache Spark ArchitectureApache Spark Architecture
Apache Spark Architecture
 
GoodFit: Multi-Resource Packing of Tasks with Dependencies
GoodFit: Multi-Resource Packing of Tasks with DependenciesGoodFit: Multi-Resource Packing of Tasks with Dependencies
GoodFit: Multi-Resource Packing of Tasks with Dependencies
 
Predictive Maintenance at the Dutch Railways with Ivo Everts
Predictive Maintenance at the Dutch Railways with Ivo EvertsPredictive Maintenance at the Dutch Railways with Ivo Everts
Predictive Maintenance at the Dutch Railways with Ivo Everts
 
Hadoop & MapReduce
Hadoop & MapReduceHadoop & MapReduce
Hadoop & MapReduce
 
Apache Spark & Streaming
Apache Spark & StreamingApache Spark & Streaming
Apache Spark & Streaming
 
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...
 
Large volume data analysis on the Typesafe Reactive Platform
Large volume data analysis on the Typesafe Reactive PlatformLarge volume data analysis on the Typesafe Reactive Platform
Large volume data analysis on the Typesafe Reactive Platform
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
 
Yahoo compares Storm and Spark
Yahoo compares Storm and SparkYahoo compares Storm and Spark
Yahoo compares Storm and Spark
 
Distributed Stream Processing - Spark Summit East 2017
Distributed Stream Processing - Spark Summit East 2017Distributed Stream Processing - Spark Summit East 2017
Distributed Stream Processing - Spark Summit East 2017
 
OpenTSDB 2.0
OpenTSDB 2.0OpenTSDB 2.0
OpenTSDB 2.0
 
Scaling Apache Storm - Strata + Hadoop World 2014
Scaling Apache Storm - Strata + Hadoop World 2014Scaling Apache Storm - Strata + Hadoop World 2014
Scaling Apache Storm - Strata + Hadoop World 2014
 
Introduction to MapReduce and Hadoop
Introduction to MapReduce and HadoopIntroduction to MapReduce and Hadoop
Introduction to MapReduce and Hadoop
 
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...
 
Spark Study Notes
Spark Study NotesSpark Study Notes
Spark Study Notes
 

Viewers also liked

Spark stream - Kafka
Spark stream - Kafka Spark stream - Kafka
Spark stream - Kafka Dori Waldman
 
Tale of Kafka Consumer for Spark Streaming
Tale of Kafka Consumer for Spark StreamingTale of Kafka Consumer for Spark Streaming
Tale of Kafka Consumer for Spark StreamingSigmoid
 
Flink Streaming Berlin Meetup
Flink Streaming Berlin MeetupFlink Streaming Berlin Meetup
Flink Streaming Berlin MeetupMárton Balassi
 
Unified Framework for Real Time, Near Real Time and Offline Analysis of Video...
Unified Framework for Real Time, Near Real Time and Offline Analysis of Video...Unified Framework for Real Time, Near Real Time and Offline Analysis of Video...
Unified Framework for Real Time, Near Real Time and Offline Analysis of Video...Spark Summit
 
Flink Apachecon Presentation
Flink Apachecon PresentationFlink Apachecon Presentation
Flink Apachecon PresentationGyula Fóra
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flinkdatamantra
 
Online learning with structured streaming, spark summit brussels 2016
Online learning with structured streaming, spark summit brussels 2016Online learning with structured streaming, spark summit brussels 2016
Online learning with structured streaming, spark summit brussels 2016Ram Sriharsha
 
Introduction to Real-time data processing
Introduction to Real-time data processingIntroduction to Real-time data processing
Introduction to Real-time data processingYogi Devendra Vyavahare
 
Rethinking Streaming Analytics For Scale
Rethinking Streaming Analytics For ScaleRethinking Streaming Analytics For Scale
Rethinking Streaming Analytics For ScaleHelena Edelson
 
Overview of Apache Flink: Next-Gen Big Data Analytics Framework
Overview of Apache Flink: Next-Gen Big Data Analytics FrameworkOverview of Apache Flink: Next-Gen Big Data Analytics Framework
Overview of Apache Flink: Next-Gen Big Data Analytics FrameworkSlim Baltagi
 
Step-by-Step Introduction to Apache Flink
Step-by-Step Introduction to Apache Flink Step-by-Step Introduction to Apache Flink
Step-by-Step Introduction to Apache Flink Slim Baltagi
 
How I got 2.5 Million views on Slideshare (by @nickdemey - Board of Innovation)
How I got 2.5 Million views on Slideshare (by @nickdemey - Board of Innovation)How I got 2.5 Million views on Slideshare (by @nickdemey - Board of Innovation)
How I got 2.5 Million views on Slideshare (by @nickdemey - Board of Innovation)Board of Innovation
 
The Seven Deadly Social Media Sins
The Seven Deadly Social Media SinsThe Seven Deadly Social Media Sins
The Seven Deadly Social Media SinsXPLAIN
 
Five Killer Ways to Design The Same Slide
Five Killer Ways to Design The Same SlideFive Killer Ways to Design The Same Slide
Five Killer Ways to Design The Same SlideCrispy Presentations
 
How People Really Hold and Touch (their Phones)
How People Really Hold and Touch (their Phones)How People Really Hold and Touch (their Phones)
How People Really Hold and Touch (their Phones)Steven Hoober
 
Upworthy: 10 Ways To Win The Internets
Upworthy: 10 Ways To Win The InternetsUpworthy: 10 Ways To Win The Internets
Upworthy: 10 Ways To Win The InternetsUpworthy
 
What 33 Successful Entrepreneurs Learned From Failure
What 33 Successful Entrepreneurs Learned From FailureWhat 33 Successful Entrepreneurs Learned From Failure
What 33 Successful Entrepreneurs Learned From FailureReferralCandy
 

Viewers also liked (20)

Spark stream - Kafka
Spark stream - Kafka Spark stream - Kafka
Spark stream - Kafka
 
Tale of Kafka Consumer for Spark Streaming
Tale of Kafka Consumer for Spark StreamingTale of Kafka Consumer for Spark Streaming
Tale of Kafka Consumer for Spark Streaming
 
Flink Streaming Berlin Meetup
Flink Streaming Berlin MeetupFlink Streaming Berlin Meetup
Flink Streaming Berlin Meetup
 
Unified Framework for Real Time, Near Real Time and Offline Analysis of Video...
Unified Framework for Real Time, Near Real Time and Offline Analysis of Video...Unified Framework for Real Time, Near Real Time and Offline Analysis of Video...
Unified Framework for Real Time, Near Real Time and Offline Analysis of Video...
 
Flink Apachecon Presentation
Flink Apachecon PresentationFlink Apachecon Presentation
Flink Apachecon Presentation
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flink
 
Online learning with structured streaming, spark summit brussels 2016
Online learning with structured streaming, spark summit brussels 2016Online learning with structured streaming, spark summit brussels 2016
Online learning with structured streaming, spark summit brussels 2016
 
Introduction to Real-time data processing
Introduction to Real-time data processingIntroduction to Real-time data processing
Introduction to Real-time data processing
 
Rethinking Streaming Analytics For Scale
Rethinking Streaming Analytics For ScaleRethinking Streaming Analytics For Scale
Rethinking Streaming Analytics For Scale
 
Overview of Apache Flink: Next-Gen Big Data Analytics Framework
Overview of Apache Flink: Next-Gen Big Data Analytics FrameworkOverview of Apache Flink: Next-Gen Big Data Analytics Framework
Overview of Apache Flink: Next-Gen Big Data Analytics Framework
 
Step-by-Step Introduction to Apache Flink
Step-by-Step Introduction to Apache Flink Step-by-Step Introduction to Apache Flink
Step-by-Step Introduction to Apache Flink
 
Flink vs. Spark
Flink vs. SparkFlink vs. Spark
Flink vs. Spark
 
The Minimum Loveable Product
The Minimum Loveable ProductThe Minimum Loveable Product
The Minimum Loveable Product
 
How I got 2.5 Million views on Slideshare (by @nickdemey - Board of Innovation)
How I got 2.5 Million views on Slideshare (by @nickdemey - Board of Innovation)How I got 2.5 Million views on Slideshare (by @nickdemey - Board of Innovation)
How I got 2.5 Million views on Slideshare (by @nickdemey - Board of Innovation)
 
The Seven Deadly Social Media Sins
The Seven Deadly Social Media SinsThe Seven Deadly Social Media Sins
The Seven Deadly Social Media Sins
 
Five Killer Ways to Design The Same Slide
Five Killer Ways to Design The Same SlideFive Killer Ways to Design The Same Slide
Five Killer Ways to Design The Same Slide
 
How People Really Hold and Touch (their Phones)
How People Really Hold and Touch (their Phones)How People Really Hold and Touch (their Phones)
How People Really Hold and Touch (their Phones)
 
Upworthy: 10 Ways To Win The Internets
Upworthy: 10 Ways To Win The InternetsUpworthy: 10 Ways To Win The Internets
Upworthy: 10 Ways To Win The Internets
 
What 33 Successful Entrepreneurs Learned From Failure
What 33 Successful Entrepreneurs Learned From FailureWhat 33 Successful Entrepreneurs Learned From Failure
What 33 Successful Entrepreneurs Learned From Failure
 
Design Your Career 2018
Design Your Career 2018Design Your Career 2018
Design Your Career 2018
 

Similar to Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms and Optimization

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
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.pptrveiga100
 
Deep dive into spark streaming
Deep dive into spark streamingDeep dive into spark streaming
Deep dive into spark streamingTao Li
 
Taking Spark Streaming to the Next Level with Datasets and DataFrames
Taking Spark Streaming to the Next Level with Datasets and DataFramesTaking Spark Streaming to the Next Level with Datasets and DataFrames
Taking Spark Streaming to the Next Level with Datasets and DataFramesDatabricks
 
Introduction to Spark Streaming
Introduction to Spark StreamingIntroduction to Spark Streaming
Introduction to Spark StreamingKnoldus Inc.
 
A Deep Dive into Structured Streaming: Apache Spark Meetup at Bloomberg 2016
A Deep Dive into Structured Streaming:  Apache Spark Meetup at Bloomberg 2016 A Deep Dive into Structured Streaming:  Apache Spark Meetup at Bloomberg 2016
A Deep Dive into Structured Streaming: Apache Spark Meetup at Bloomberg 2016 Databricks
 
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
 
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
 
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
 
Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...Databricks
 
Metadata and Provenance for ML Pipelines with Hopsworks
Metadata and Provenance for ML Pipelines with Hopsworks Metadata and Provenance for ML Pipelines with Hopsworks
Metadata and Provenance for ML Pipelines with Hopsworks Jim Dowling
 
Dataservices: Processing (Big) Data the Microservice Way
Dataservices: Processing (Big) Data the Microservice WayDataservices: Processing (Big) Data the Microservice Way
Dataservices: Processing (Big) Data the Microservice WayQAware GmbH
 
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
 
Kafka Spark Realtime stream processing and analytics in 6 steps
Kafka Spark Realtime stream processing and analytics in 6 stepsKafka Spark Realtime stream processing and analytics in 6 steps
Kafka Spark Realtime stream processing and analytics in 6 stepsAzmath Mohamad
 
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
 

Similar to Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms and Optimization (20)

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
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.ppt
 
Spark streaming
Spark streamingSpark streaming
Spark streaming
 
Deep dive into spark streaming
Deep dive into spark streamingDeep dive into spark streaming
Deep dive into spark streaming
 
Taking Spark Streaming to the Next Level with Datasets and DataFrames
Taking Spark Streaming to the Next Level with Datasets and DataFramesTaking Spark Streaming to the Next Level with Datasets and DataFrames
Taking Spark Streaming to the Next Level with Datasets and DataFrames
 
Spark streaming
Spark streamingSpark streaming
Spark streaming
 
Introduction to Spark Streaming
Introduction to Spark StreamingIntroduction to Spark Streaming
Introduction to Spark Streaming
 
A Deep Dive into Structured Streaming: Apache Spark Meetup at Bloomberg 2016
A Deep Dive into Structured Streaming:  Apache Spark Meetup at Bloomberg 2016 A Deep Dive into Structured Streaming:  Apache Spark Meetup at Bloomberg 2016
A Deep Dive into Structured Streaming: Apache Spark Meetup at Bloomberg 2016
 
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...
 
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
 
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)
 
Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...Writing Continuous Applications with Structured Streaming Python APIs in Apac...
Writing Continuous Applications with Structured Streaming Python APIs in Apac...
 
Metadata and Provenance for ML Pipelines with Hopsworks
Metadata and Provenance for ML Pipelines with Hopsworks Metadata and Provenance for ML Pipelines with Hopsworks
Metadata and Provenance for ML Pipelines with Hopsworks
 
Dataservices: Processing (Big) Data the Microservice Way
Dataservices: Processing (Big) Data the Microservice WayDataservices: Processing (Big) Data the Microservice Way
Dataservices: Processing (Big) Data the Microservice Way
 
PuttingItAllTogether
PuttingItAllTogetherPuttingItAllTogether
PuttingItAllTogether
 
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)
 
Kafka Spark Realtime stream processing and analytics in 6 steps
Kafka Spark Realtime stream processing and analytics in 6 stepsKafka Spark Realtime stream processing and analytics in 6 steps
Kafka Spark Realtime stream processing and analytics in 6 steps
 
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...
 

Recently uploaded

Dubai Call Girls Wifey O52&786472 Call Girls Dubai
Dubai Call Girls Wifey O52&786472 Call Girls DubaiDubai Call Girls Wifey O52&786472 Call Girls Dubai
Dubai Call Girls Wifey O52&786472 Call Girls Dubaihf8803863
 
Customer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptxCustomer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptxEmmanuel Dauda
 
FESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfFESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfMarinCaroMartnezBerg
 
Call Girls In Mahipalpur O9654467111 Escorts Service
Call Girls In Mahipalpur O9654467111  Escorts ServiceCall Girls In Mahipalpur O9654467111  Escorts Service
Call Girls In Mahipalpur O9654467111 Escorts ServiceSapana Sha
 
Digi Khata Problem along complete plan.pptx
Digi Khata Problem along complete plan.pptxDigi Khata Problem along complete plan.pptx
Digi Khata Problem along complete plan.pptxTanveerAhmed817946
 
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service BhilaiLow Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service BhilaiSuhani Kapoor
 
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdfMarket Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdfRachmat Ramadhan H
 
Unveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data AnalystUnveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data AnalystSamantha Rae Coolbeth
 
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfKantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfSocial Samosa
 
100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptxAnupama Kate
 
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...dajasot375
 
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...Suhani Kapoor
 
RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998YohFuh
 
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...Jack DiGiovanna
 
Beautiful Sapna Vip Call Girls Hauz Khas 9711199012 Call /Whatsapps
Beautiful Sapna Vip  Call Girls Hauz Khas 9711199012 Call /WhatsappsBeautiful Sapna Vip  Call Girls Hauz Khas 9711199012 Call /Whatsapps
Beautiful Sapna Vip Call Girls Hauz Khas 9711199012 Call /Whatsappssapnasaifi408
 
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改atducpo
 
Aminabad Call Girl Agent 9548273370 , Call Girls Service Lucknow
Aminabad Call Girl Agent 9548273370 , Call Girls Service LucknowAminabad Call Girl Agent 9548273370 , Call Girls Service Lucknow
Aminabad Call Girl Agent 9548273370 , Call Girls Service Lucknowmakika9823
 
Industrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfIndustrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfLars Albertsson
 
定制英国白金汉大学毕业证(UCB毕业证书) 成绩单原版一比一
定制英国白金汉大学毕业证(UCB毕业证书)																			成绩单原版一比一定制英国白金汉大学毕业证(UCB毕业证书)																			成绩单原版一比一
定制英国白金汉大学毕业证(UCB毕业证书) 成绩单原版一比一ffjhghh
 

Recently uploaded (20)

Dubai Call Girls Wifey O52&786472 Call Girls Dubai
Dubai Call Girls Wifey O52&786472 Call Girls DubaiDubai Call Girls Wifey O52&786472 Call Girls Dubai
Dubai Call Girls Wifey O52&786472 Call Girls Dubai
 
Customer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptxCustomer Service Analytics - Make Sense of All Your Data.pptx
Customer Service Analytics - Make Sense of All Your Data.pptx
 
FESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdfFESE Capital Markets Fact Sheet 2024 Q1.pdf
FESE Capital Markets Fact Sheet 2024 Q1.pdf
 
Call Girls In Mahipalpur O9654467111 Escorts Service
Call Girls In Mahipalpur O9654467111  Escorts ServiceCall Girls In Mahipalpur O9654467111  Escorts Service
Call Girls In Mahipalpur O9654467111 Escorts Service
 
Digi Khata Problem along complete plan.pptx
Digi Khata Problem along complete plan.pptxDigi Khata Problem along complete plan.pptx
Digi Khata Problem along complete plan.pptx
 
Decoding Loan Approval: Predictive Modeling in Action
Decoding Loan Approval: Predictive Modeling in ActionDecoding Loan Approval: Predictive Modeling in Action
Decoding Loan Approval: Predictive Modeling in Action
 
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service BhilaiLow Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
Low Rate Call Girls Bhilai Anika 8250192130 Independent Escort Service Bhilai
 
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdfMarket Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
Market Analysis in the 5 Largest Economic Countries in Southeast Asia.pdf
 
Unveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data AnalystUnveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data Analyst
 
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfKantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
 
100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx
 
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
 
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
 
RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998RA-11058_IRR-COMPRESS Do 198 series of 1998
RA-11058_IRR-COMPRESS Do 198 series of 1998
 
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
Building on a FAIRly Strong Foundation to Connect Academic Research to Transl...
 
Beautiful Sapna Vip Call Girls Hauz Khas 9711199012 Call /Whatsapps
Beautiful Sapna Vip  Call Girls Hauz Khas 9711199012 Call /WhatsappsBeautiful Sapna Vip  Call Girls Hauz Khas 9711199012 Call /Whatsapps
Beautiful Sapna Vip Call Girls Hauz Khas 9711199012 Call /Whatsapps
 
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
代办国外大学文凭《原版美国UCLA文凭证书》加州大学洛杉矶分校毕业证制作成绩单修改
 
Aminabad Call Girl Agent 9548273370 , Call Girls Service Lucknow
Aminabad Call Girl Agent 9548273370 , Call Girls Service LucknowAminabad Call Girl Agent 9548273370 , Call Girls Service Lucknow
Aminabad Call Girl Agent 9548273370 , Call Girls Service Lucknow
 
Industrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfIndustrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdf
 
定制英国白金汉大学毕业证(UCB毕业证书) 成绩单原版一比一
定制英国白金汉大学毕业证(UCB毕业证书)																			成绩单原版一比一定制英国白金汉大学毕业证(UCB毕业证书)																			成绩单原版一比一
定制英国白金汉大学毕业证(UCB毕业证书) 成绩单原版一比一
 

Guest Lecture on Spark Streaming in Stanford CME 323: Distributed Algorithms and Optimization

  • 1. Streaming items through a cluster with Spark Streaming Tathagata “TD” Das @tathadas CME 323: Distributed Algorithms and Optimization Stanford, May 6, 2015
  • 2. Who am I? > Project Management Committee (PMC) member of Apache Spark > Lead developer of Spark Streaming > Formerly in AMPLab, UC Berkeley > Software developer at Databricks > Databricks was started by creators of Spark to provide Spark-as-a-service in the cloud
  • 5. Why process Big Streaming Data? Fraud detection in bank transactions Anomalies in sensor data Cat videos in tweets
  • 6. How to Process Big Streaming Data > Ingest – Receive and buffer the streaming data > Process – Clean, extract, transform the data > Store – Store transformed data for consumption Ingest data Process data Store results Raw Tweets
  • 7. How to Process Big Streaming Data > For big streams, every step requires a cluster > Every step requires a system that is designed for it Ingest data Process data Store results Raw Tweets
  • 8. Stream Ingestion Systems > Kafka – popular distributed pub-sub system > Kinesis – Amazon managed distributed pub-sub > Flume – like a distributed data pipe Ingest data Process data Store results Raw Tweets Amazon Kinesis
  • 9. Stream Ingestion Systems > Spark Streaming – From UC Berkeley + Databricks > Storm – From Backtype + Twitter > Samza – From LinkedIn Ingest data Process data Store results Raw Tweets
  • 10. Stream Ingestion Systems > File systems – HDFS, Amazon S3, etc. > Key-value stores – HBase, Cassandra, etc. > Databases – MongoDB, MemSQL, etc. Ingest data Process data Store results Raw Tweets
  • 11.
  • 12. Kafka Cluster Producers and Consumers > Producers publish data tagged by “topic” > Consumers subscribe to data of a particular “topic” Producer 1 Producer 2 (topicX, data1) (topicY, data2) (topicX, data3) Topic X Consumer Topic Y Consumer (topicX, data1) (topicX, data3) (topicY, data2)
  • 13. > Topic = category of message, divided into partitions > Partition = ordered, numbered stream of messages > Producer decides which (topic, partition) to put each message in Topics and Partitions
  • 14. > Topic = category of message, divided into partitions > Partition = ordered, numbered stream of messages > Producer decides which (topic, partition) to put each message in > Consumer decides which (topic, partition) to pull messages from - High-level consumer – handles fault-recovery with Zookeeper - Simple consumer – low-level API for greater control Topics and Partitions
  • 15. How to process Kafka messages? Ingest data Process data Store results Raw Tweets > Incoming tweets received in distributed manner and buffered in Kafka > How to process them?
  • 17. What is Spark Streaming? Scalable, fault-tolerant stream processing system File systems Databases Dashboards Flume HDFS Kinesis Kafka Twitter High-level API joins, windows, … often 5x less code Fault-tolerant Exactly-once semantics, even for stateful ops Integration Integrate with MLlib, SQL, DataFrames, GraphX
  • 18. How does Spark Streaming work? > Receivers chop up data streams into batches of few seconds > Spark processing engine processes each batch and pushes out the results to external data stores data streams Receivers batches as RDDs results as RDDs
  • 19. Spark Programming Model > Resilient distributed datasets (RDDs) - Distributed, partitioned collection of objects - Manipulated through parallel transformations (map, filter, reduceByKey, …) - All transformations are lazy, execution forced by actions (count, reduce, take, …) - Can be cached in memory across cluster - Automatically rebuilt on failure
  • 20. Spark Streaming Programming Model > Discretized Stream (DStream) - Represents a stream of data - Implemented as a infinite sequence of RDDs > DStreams API very similar to RDD API - Functional APIs in - Create input DStreams from Kafka, Flume, Kinesis, HDFS, … - Apply transformations
  • 21. Example – Get hashtags from Twitter val ssc = new StreamingContext(conf, Seconds(1)) StreamingContext is the starting point of all streaming functionality Batch interval, by which streams will be chopped up
  • 22. Example – Get hashtags from Twitter val ssc = new StreamingContext(conf, Seconds(1)) val tweets = TwitterUtils.createStream(ssc, auth) Input DStream batch @ t+1batch @ t batch @ t+2 tweets DStream replicated and stored in memory as RDDs Twitter Streaming API
  • 23. Example – Get hashtags from Twitter val tweets = TwitterUtils.createStream(ssc, None) val hashTags = tweets.flatMap(status => getTags(status)) flatMap flatMap flatMap … transformation: modify data in one DStream to create another DStream transformed DStream new RDDs created for every batch batch @ t+1batch @ t batch @ t+2 tweets DStream hashTags Dstream [#cat, #dog, … ]
  • 24. Example – Get hashtags from Twitter val tweets = TwitterUtils.createStream(ssc, None) val hashTags = tweets.flatMap(status => getTags(status)) hashTags.saveAsTextFiles("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
  • 25. Example – Get hashtags from Twitter val tweets = TwitterUtils.createStream(ssc, None) val hashTags = tweets.flatMap(status => getTags(status)) hashTags.foreachRDD(hashTagRDD => { ... }) foreachRDD: do whatever you want with the processed data flatMap flatMap flatMap foreach foreach foreach batch @ t+1batch @ t batch @ t+2 tweets DStream hashTags DStream Write to a database, update analytics UI, do whatever you want
  • 26. Example – Get hashtags from Twitter val tweets = TwitterUtils.createStream(ssc, None) val hashTags = tweets.flatMap(status => getTags(status)) hashTags.foreachRDD(hashTagRDD => { ... }) ssc.start() all of this was just setup for what to do when streaming data is receiver this actually starts the receiving and processing
  • 27. What’s going on inside? > Receiver buffers tweets in Executors’ memory > Spark Streaming Driver launches tasks to process tweets Driver running DStreams Executors launch tasks to process tweets Raw Tweets Twitter Receiver Buffered Tweets Buffered Tweets Spark Cluster
  • 28. What’s going on inside? Driver running DStreams Executors launch tasks to process data Receiver Kafka Cluster ReceiverReceiver receive data in parallel Spark Cluster
  • 29. Performance Can process 60M records/sec (6 GB/sec) on 100 nodes at sub-second latency 0 0.5 1 1.5 2 2.5 3 3.5 0 50 100 ClusterThroughput(GB/s) # Nodes in Cluster WordCount 1 sec 2 sec 0 1 2 3 4 5 6 7 0 50 100 ClusterThhroughput(GB/s) # Nodes in Cluster Grep 1 sec 2 sec
  • 30. DStream of data Window-based Transformations val tweets = TwitterUtils.createStream(ssc, auth) val hashTags = tweets.flatMap(status => getTags(status)) val tagCounts = hashTags.window(Minutes(1), Seconds(5)).countByValue() sliding window operation window length sliding interval window length sliding interval
  • 31. Arbitrary Stateful Computations Specify function to generate new state based on previous state and new data - Example: Maintain per-user mood as state, and update it with their tweets def updateMood(newTweets, lastMood) => newMood val moods = tweetsByUser.updateStateByKey(updateMood _)
  • 32. Integrates with Spark Ecosystem Spark Core Spark Streaming Spark SQL MLlib GraphX
  • 33. Combine batch and streaming processing > Join data streams with static data sets // Create data set from Hadoop file val dataset = sparkContext.hadoopFile(“file”) // Join each batch in stream with dataset kafkaStream.transform { batchRDD => batchRDD.join(dataset)filter(...) } Spark Core Spark Streaming Spark SQL MLlib GraphX
  • 34. Combine machine learning with streaming > Learn models offline, apply them online // Learn model offline val model = KMeans.train(dataset, ...) // Apply model online on stream kafkaStream.map { event => model.predict(event.feature) } Spark Core Spark Streaming Spark SQL MLlib GraphX
  • 35. Combine SQL with streaming > Interactively query streaming data with SQL // Register each batch in stream as table kafkaStream.map { batchRDD => batchRDD.registerTempTable("latestEvents") } // Interactively query table sqlContext.sql("select * from latestEvents") Spark Core Spark Streaming Spark SQL MLlib GraphX
  • 37. Why are they adopting Spark Streaming? Easy, high-level API Unified API across batch and streaming Integration with Spark SQL and MLlib Ease of operations 37
  • 38. Neuroscience @ Freeman Lab, Janelia Farm Spark Streaming and MLlib to analyze neural activities Laser microscope scans Zebrafish brain Spark Streaming  interactive visualization  laser ZAP to kill neurons! http://www.jeremyfreeman.net/share/talks/spark-summit-2014/
  • 39. Neuroscience @ Freeman Lab, Janelia Farm Streaming machine learning algorithms on time series data of every neuron Upto 2TB/hour and increasing with brain size Upto 80 HPC nodes http://www.jeremyfreeman.net/share/talks/spark-summit-2014/
  • 40. Streaming Machine Learning Algos > Streaming Linear Regression > Streaming Logistic Regression > Streaming KMeans http://www.jeremyfreeman.net/share/talks/spark-summit-east-2015/#/algorithms-repeat
  • 41. Okay okay, how do I start off? > Online Streaming Programming Guide http://spark.apache.org/docs/latest/streaming-programming-guide.html > Streaming examples https://github.com/apache/spark/tree/master/examples/src/main/scala/o rg/apache/spark/examples/streaming