SlideShare a Scribd company logo
1 of 62
Download to read offline
Spark Streaming Case Studies 
Strata EU, Barcelona 
2014-11-20 
strataconf.com/strataeu2014/public/schedule/detail/37493 
Paco Nathan 
@pacoid
Spark, 
the elevator pitch 
2
Spark, the elevator pitch 
Developed in 2009 at UC Berkeley AMPLab, 
open sourced in 2010, Spark has since become 
one of the largest OSS communities in big data, 
with over 200 contributors in 50+ organizations 
spark.apache.org 
“Organizations that are looking at big data challenges – 
including collection, ETL, storage, exploration and analytics – 
should consider Spark for its in-memory performance and 
the breadth of its model. 
It supports advanced analytics solutions on Hadoop 
clusters, including the iterative model required for 
machine learning and graph analysis.” 
Gartner, Advanced Analytics and Data Science (2014) 
3
Spark, the elevator pitch 
4
circa 2010: 
a unified engine for enterprise data workflows, 
based on commodity hardware a decade later… 
Spark: Cluster Computing with Working Sets 
Matei Zaharia, Mosharaf Chowdhury, 
Michael Franklin, Scott Shenker, Ion Stoica 
people.csail.mit.edu/matei/papers/2010/hotcloud_spark.pdf 
! 
Resilient Distributed Datasets: A Fault-Tolerant Abstraction for 
In-Memory Cluster Computing 
Matei Zaharia, Mosharaf Chowdhury, Tathagata Das, Ankur Dave, 
Justin Ma, Murphy McCauley, Michael Franklin, Scott Shenker, Ion Stoica 
usenix.org/system/files/conference/nsdi12/nsdi12-final138.pdf 
5 
Spark, the elevator pitch
Spark, the elevator pitch 
Spark Core is the general execution engine for the 
Spark platform that other functionality is built atop: 
! 
• in-memory computing capabilities deliver speed 
• general execution model supports wide variety 
of use cases 
• ease of development – native APIs in Java, Scala, 
Python (+ SQL, Clojure, R) 
6
WordCount in 3 lines of Spark 
Spark, the elevator pitch 
WordCount in 50+ lines of Java MR 
7
Spark, the elevator pitch 
Sustained exponential growth, as one of the most 
active Apache projects ohloh.net/orgs/apache 
8
TL;DR: Smashing The Previous Petabyte Sort Record 
databricks.com/blog/2014/11/05/spark-officially-sets- 
a-new-record-in-large-scale-sorting.html 
9
Why Streaming? 
10
Why Streaming? 
Because Machine Data! 
I <3 Logs 
Jay Kreps 
O’Reilly (2014) 
shop.oreilly.com/product/ 
0636920034339.do 
11
Why Streaming? 
Because Google! 
MillWheel: Fault-Tolerant Stream 
Processing at Internet Scale 
Tyler Akidau, Alex Balikov, 
Kaya Bekiroglu, Slava Chernyak, 
Josh Haberman, Reuven Lax, 
Sam McVeety, Daniel Mills, 
Paul Nordstrom, Sam Whittle 
Very Large Data Bases (2013) 
research.google.com/pubs/ 
pub41378.html 
12
Why Streaming? 
Because IoT! 
kickstarter.com/projects/1614456084/b4rm4n-be- 
a-cocktail-hero 
13
Why Streaming? 
Because IoT! (exabytes/day per sensor) 
bits.blogs.nytimes.com/2013/06/19/g-e-makes-the-machine- 
and-then-uses-sensors-to-listen-to-it/ 
14
Spark Streaming 
15
Spark Streaming: Requirements 
Let’s consider the top-level requirements for 
a streaming framework: 
• clusters scalable to 100’s of nodes 
• low-latency, in the range of seconds 
(meets 90% of use case needs) 
• efficient recovery from failures 
(which is a hard problem in CS) 
• integrates with batch: many co’s run the 
same business logic both online+offline 
16
Spark Streaming: Requirements 
Therefore, run a streaming computation as: 
a series of very small, deterministic batch jobs 
! 
• 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 
17
Spark Streaming: Requirements 
Therefore, run a streaming computation as: 
a series of very small, deterministic batch jobs 
! 
• Batch sizes as low as ½ sec, 
latency of about 1 sec 
• Potential for combining 
batch processing and 
streaming processing in 
the same system 
18
Spark Streaming: Integration 
Data can be ingested from many sources: 
Kafka, Flume, Twitter, ZeroMQ, TCP sockets, etc. 
Results can be pushed out to filesystems, 
databases, live dashboards, etc. 
Spark’s built-in machine learning algorithms and 
graph processing algorithms can be applied to 
data streams 
19
Spark Streaming: Timeline 
2012 
project started 
2013 
alpha release (Spark 0.7) 
2014 
graduated (Spark 0.9) 
Discretized Streams: A Fault-Tolerant Model 
for Scalable Stream Processing 
Matei Zaharia, Tathagata Das, Haoyuan Li, 
Timothy Hunter, Scott Shenker, Ion Stoica 
Berkeley EECS (2012-12-14) 
www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf 
project lead: 
Tathagata Das @tathadas 
20
Spark Streaming: Requirements 
Typical kinds of applications: 
• datacenter operations 
• web app funnel metrics 
• ad optimization 
• anti-fraud 
• various telematics 
and much much more! 
21
Spark Streaming: Some Excellent Resources 
Programming Guide 
spark.apache.org/docs/latest/streaming-programming- 
guide.html 
TD @ Spark Summit 2014 
youtu.be/o-NXwFrNAWQ?list=PLTPXxbhUt- 
YWGNTaDj6HSjnHMxiTD1HCR 
“Deep Dive into Spark Streaming” 
slideshare.net/spark-project/deep-divewithsparkstreaming-tathagatadassparkmeetup20130617 
Spark Reference Applications 
databricks.gitbooks.io/databricks-spark-reference- 
applications/ 
22
Quiz: name the bits and pieces… 
import org.apache.spark.streaming._! 
import org.apache.spark.streaming.StreamingContext._! 
! 
// create a StreamingContext with a SparkConf configuration! 
val ssc = new StreamingContext(sparkConf, Seconds(10))! 
! 
// create a DStream that will connect to serverIP:serverPort! 
val lines = ssc.socketTextStream(serverIP, serverPort)! 
! 
// split each line into words! 
val words = lines.flatMap(_.split(" "))! 
! 
// count each word in each batch! 
val pairs = words.map(word => (word, 1))! 
val wordCounts = pairs.reduceByKey(_ + _)! 
! 
// print a few of the counts to the console! 
wordCounts.print()! 
! 
ssc.start()! 
ssc.awaitTermination() 
23
Because 
Use Cases 
24
Because Use Cases: +40 known production use cases 
25
Because Use Cases: Analysis 
Reasons for adopting/transitioning to Spark 
Streaming… the unified programming model is 
particularly relevant for real-time analytics that 
combine historical data: 
• Making data science accessible to non-scientists 
• Higher productivity for data workers 
• Exactly-once semantics 
• No compromises on scalability and throughput 
• Ease of operations 
26
Because Use Cases: Stratio 
Stratio Streaming: a new approach to 
Spark Streaming 
David Morales, Oscar Mendez 
2014-06-30 
spark-summit.org/2014/talk/stratio-streaming- 
a-new-approach-to-spark-streaming 
• Stratio Streaming is the union of a real-time 
messaging bus with a complex event processing 
engine using Spark Streaming 
• allows the creation of streams and queries on the fly 
• paired with Siddhi CEP engine and Apache Kafka 
• added global features to the engine such as auditing 
27 
and statistics
Because Use Cases: Ooyala 
Productionizing a 24/7 Spark Streaming 
service on YARN 
Issac Buenrostro, Arup Malakar 
2014-06-30 
spark-summit.org/2014/talk/ 
productionizing-a-247-spark-streaming-service- 
on-yarn 
• state-of-the-art ingestion pipeline, processing over 
two billion video events a day 
• how do you ensure 24/7 availability and fault 
tolerance? 
• what are the best practices for Spark Streaming and 
its integration with Kafka and YARN? 
• how do you monitor and instrument the various 
28 
stages of the pipeline?
Because Use Cases: Guavus 
Guavus Embeds Apache Spark 
into its Operational Intelligence Platform 
Deployed at the World’s Largest Telcos 
Eric Carr 
2014-09-25 
databricks.com/blog/2014/09/25/guavus-embeds-apache-spark-into- 
its-operational-intelligence-platform-deployed-at-the-worlds- 
largest-telcos.html 
• 4 of 5 top mobile network operators, 3 of 5 top 
Internet backbone providers, 80% MSOs in NorAm 
• analyzing 50% of US mobile data traffic, +2.5 PB/day 
• latency is critical for resolving operational issues 
before they cascade: 2.5 MM transactions per second 
• “analyze first” not “store first ask questions later” 
29
Because Use Cases: Sharethrough 
Sharethrough Uses Spark Streaming to 
Optimize Bidding in Real Time 
Russell Cardullo, Michael Ruggier 
2014-03-25 
databricks.com/blog/2014/03/25/ 
sharethrough-and-spark-streaming.html 
• the profile of a 24 x 7 streaming app is different than 
an hourly batch job… 
• take time to validate output against the input… 
• confirm that supporting objects are being serialized… 
• the output of your Spark Streaming job is only as 
reliable as the queue that feeds Spark… 
• monoids… 
30
Because Use Cases: Viadeo 
Spark Streaming As Near Realtime ETL 
Djamel Zouaoui 
2014-09-18 
slideshare.net/DjamelZouaoui/spark-streaming 
• Spark Streaming is topology-free 
• workers and receivers are autonomous and 
independent 
• paired with Kafka, RabbitMQ 
• 8 machines / 120 cores 
• use case for recommender system 
• issues: how to handle lost data, serialization 
31
Demos 
32
Demos: 
brand new Python support for Streaming in 1.2 
github.com/apache/spark/tree/master/examples/src/main/ 
python/streaming 
! 
For more Spark learning resources online: 
databricks.com/spark-training-resources 
33
Demo: PySpark Streaming Network Word Count 
import sys! 
from pyspark import SparkContext! 
from pyspark.streaming import StreamingContext! 
! 
sc = SparkContext(appName="PyStreamNWC", master="local[*]")! 
ssc = StreamingContext(sc, Seconds(5))! 
! 
lines = ssc.socketTextStream(sys.argv[1], int(sys.argv[2]))! 
! 
counts = lines.flatMap(lambda line: line.split(" ")) ! 
.map(lambda word: (word, 1)) ! 
.reduceByKey(lambda a, b: a+b)! 
! 
counts.pprint()! 
! 
ssc.start()! 
ssc.awaitTermination() 
34
Demo: PySpark Streaming Network Word Count - Stateful 
import sys! 
from pyspark import SparkContext! 
from pyspark.streaming import StreamingContext! 
! 
def updateFunc (new_values, last_sum):! 
return sum(new_values) + (last_sum or 0)! 
! 
sc = SparkContext(appName="PyStreamNWC", master="local[*]")! 
ssc = StreamingContext(sc, Seconds(5))! 
ssc.checkpoint("checkpoint")! 
! 
lines = ssc.socketTextStream(sys.argv[1], int(sys.argv[2]))! 
! 
counts = lines.flatMap(lambda line: line.split(" ")) ! 
.map(lambda word: (word, 1)) ! 
.updateStateByKey(updateFunc) ! 
.transform(lambda x: x.sortByKey())! 
! 
counts.pprint()! 
! 
ssc.start()! 
ssc.awaitTermination() 
35
Complementary 
Frameworks 
36
Spark Integrations: 
Discover 
Insights 
Clean Up 
Your Data 
Run 
Sophisticated 
Analytics 
Integrate With 
Many Other 
Systems 
Use Lots of Different 
Data Sources 
cloud-based notebooks… ETL… the Hadoop ecosystem… 
widespread use of PyData… advanced analytics in streaming… 
rich custom search… web apps for data APIs… 
low-latency + multi-tenancy… 
37
Spark Integrations: Advanced analytics for streaming use cases 
Kafka + Spark + Cassandra 
datastax.com/documentation/datastax_enterprise/4.5/ 
datastax_enterprise/spark/sparkIntro.html 
http://helenaedelson.com/?p=991 
github.com/datastax/spark-cassandra-connector 
github.com/dibbhatt/kafka-spark-consumer 
unified compute 
data streams columnar key-value 
38
Spark Integrations: Rich search, immediate insights 
Spark + ElasticSearch 
databricks.com/blog/2014/06/27/application-spotlight-elasticsearch. 
html 
elasticsearch.org/guide/en/elasticsearch/hadoop/current/ 
spark.html 
spark-summit.org/2014/talk/streamlining-search-indexing- 
using-elastic-search-and-spark 
unified compute 
document search 
39
Spark Integrations: General Guidelines 
• use Tachyon as a best practice for sharing 
between two streaming apps 
• or write to Cassandra or HBase / then read back 
• design patterns for integration: 
spark.apache.org/docs/latest/streaming-programming- 
guide.html#output-operations- 
on-dstreams 
40
A Look Ahead… 
41
A Look Ahead… 
1. Greater Stability and Robustness 
• improved high availability via write-ahead logs 
• enabled as an optional feature for Spark 1.2 
• NB: Spark Standalone can already restart driver 
• excellent discussion of fault-tolerance (2012): 
cs.duke.edu/~kmoses/cps516/dstream.html 
• stay tuned: 
meetup.com/spark-users/events/218108702/ 
42
A Look Ahead… 
2. Support for more environments, 
i.e., beyond Hadoop 
• three use cases currently depend on HDFS 
• those are being abstracted out 
• could then use Cassandra, etc. 
43
A Look Ahead… 
3. Improved support for Python 
• e.g., Kafka is not exposed through Python yet 
(next release goal) 
44
A Look Ahead… 
4. Better flow control 
• a somewhat longer-term goal, plus it is 
a hard problem in general 
• poses interesting challenges beyond what 
other streaming systems have faced 
45
A Big Picture 
46
A Big Picture… 
19-20c. statistics emphasized defensibility 
in lieu of predictability, based on analytic 
variance and goodness-of-fit tests 
! 
That approach inherently led toward a 
manner of computational thinking based 
on batch windows 
! 
They missed a subtle point… 
47
A Big Picture… The view in the lens has changed 
21c. shift towards modeling based on probabilistic 
approximations: trade bounded errors for greatly 
reduced resource costs 
highlyscalable.wordpress.com/2012/05/01/ 
probabilistic-structures-web-analytics-data- 
mining/ 
48
A Big Picture… The view in the lens has changed 
21c. shift towards modeling based on probabil 
approximations: trade bounded errors for greatly 
reduced resource costs 
Twitter catch-phrase: 
“Hash, don’t sample” 
highlyscalable.wordpress.com/2012/05/01/ 
probabilistic-structures-web-analytics-data- 
mining/ 
49
Probabilistic Data Structures: 
a fascinating and relatively new area, pioneered 
by relatively few people – e.g., Philippe Flajolet 
provides approximation, with error bounds – 
in general uses significantly less resources 
(RAM, CPU, etc.) 
many algorithms can be constructed from 
combinations of read and write monoids 
aggregate different ranges by composing 
hashes, instead of repeating full-queries 
50
Probabilistic Data Structures: Some Examples 
algorithm use case example 
Count-Min Sketch frequency summaries code 
HyperLogLog set cardinality code 
Bloom Filter set membership 
MinHash 
set similarity 
DSQ streaming quantiles 
SkipList ordered sequence search 
51
Probabilistic Data Structures: Some Examples 
algorithm use case example 
Count-Min Sketch frequency summaries code 
HyperLogLog set cardinality code 
suggestion: consider these 
as your most quintessential 
collections data types at scale 
Bloom Filter set membership 
MinHash 
set similarity 
DSQ streaming quantiles 
SkipList ordered sequence search 
52
Add ALL the Things: 
Abstract Algebra Meets Analytics 
infoq.com/presentations/abstract-algebra-analytics 
Avi Bryant, Strange Loop (2013) 
• grouping doesn’t matter (associativity) 
• ordering doesn’t matter (commutativity) 
• zeros get ignored 
In other words, while partitioning data at 
scale is quite difficult, you can let the math 
allow your code to be flexible at scale 
Avi Bryant 
@avibryant 
Probabilistic Data Structures: Performance Bottlenecks 
53
Probabilistic Data Structures: Industry Drivers 
• sketch algorithms: trade bounded errors for 
orders of magnitude less required resources, 
e.g., fit more complex apps in memory 
• multicore + large memory spaces (off heap) are 
increasing the resources per node in a cluster 
• containers allow for finer-grain allocation of 
cluster resources and multi-tenancy 
• monoids, etc.: guarantees of associativity within 
the code allow for more effective distributed 
computing, e.g., partial aggregates 
• less resources must be spent sorting/windowing 
data prior to working with a data set 
• real-time apps, which don’t have the luxury of 
anticipating data partitions, can respond quickly 
54
Probabilistic Data Structures: Recommended Reading 
Probabilistic Data Structures for Web 
Analytics and Data Mining 
Ilya Katsov (2012-05-01) 
A collection of links for streaming 
algorithms and data structures 
Debasish Ghosh 
Aggregate Knowledge blog (now Neustar) 
Timon Karnezos, Matt Curcio, et al. 
Probabilistic Data Structures and 
Breaking Down Big Sequence Data 
C. Titus Brown, O'Reilly (2010-11-10) 
Algebird 
Avi Bryant, Oscar Boykin, et al. Twitter (2012) 
Mining of Massive Datasets 
Jure Leskovec, Anand Rajaraman, 
Jeff Ullman, Cambridge (2011) 
55
Resources 
56
cloud-based notebooks: 
databricks.com/blog/2014/07/14/databricks-cloud-making- 
big-data-easy.html 
youtube.com/watch?v=dJQ5lV5Tldw#t=883 
57
certification: 
Apache Spark developer certificate program 
• http://oreilly.com/go/sparkcert 
• defined by Spark experts @Databricks 
• assessed by O’Reilly Media 
• establishes the bar for Spark expertise 
58
community: 
spark.apache.org/community.html 
video+slide archives: spark-summit.org 
events worldwide: goo.gl/2YqJZK 
resources: databricks.com/spark-training-resources 
workshops: databricks.com/spark-training 
59
books: 
Fast Data Processing 
with Spark 
Holden Karau 
Packt (2013) 
shop.oreilly.com/product/ 
9781782167068.do 
Spark in Action 
Chris Fregly 
Manning (2015*) 
sparkinaction.com/ 
Learning Spark 
Holden Karau, 
Andy Konwinski, 
Matei Zaharia 
O’Reilly (2015*) 
shop.oreilly.com/product/ 
0636920028512.do 
60
events: 
Strata EU 
Barcelona, Nov 19-21 
strataconf.com/strataeu2014 
Data Day Texas 
Austin, Jan 10 
datadaytexas.com 
Strata CA 
San Jose, Feb 18-20 
strataconf.com/strata2015 
Spark Summit East 
NYC, Mar 18-19 
spark-summit.org/east 
Strata EU 
London, May 5-7 
strataconf.com/big-data-conference-uk-2015 
Spark Summit 2015 
SF, Jun 15-17 
spark-summit.org 
61
presenter: 
monthly newsletter for updates, 
events, conf summaries, etc.: 
liber118.com/pxn/ 
Just Enough Math 
O’Reilly, 2014 
justenoughmath.com 
preview: youtu.be/TQ58cWgdCpA 
Enterprise Data Workflows 
with Cascading 
O’Reilly, 2013 
shop.oreilly.com/product/ 
0636920028536.do 62

More Related Content

What's hot

CERN’s Next Generation Data Analysis Platform with Apache Spark with Enric Te...
CERN’s Next Generation Data Analysis Platform with Apache Spark with Enric Te...CERN’s Next Generation Data Analysis Platform with Apache Spark with Enric Te...
CERN’s Next Generation Data Analysis Platform with Apache Spark with Enric Te...
Databricks
 
Microservices, containers, and machine learning
Microservices, containers, and machine learningMicroservices, containers, and machine learning
Microservices, containers, and machine learning
Paco Nathan
 

What's hot (20)

Data Science at Scale by Sarah Guido
Data Science at Scale by Sarah GuidoData Science at Scale by Sarah Guido
Data Science at Scale by Sarah Guido
 
Apache Flink: Real-World Use Cases for Streaming Analytics
Apache Flink: Real-World Use Cases for Streaming AnalyticsApache Flink: Real-World Use Cases for Streaming Analytics
Apache Flink: Real-World Use Cases for Streaming Analytics
 
Kafka for data scientists
Kafka for data scientistsKafka for data scientists
Kafka for data scientists
 
Why apache Flink is the 4G of Big Data Analytics Frameworks
Why apache Flink is the 4G of Big Data Analytics FrameworksWhy apache Flink is the 4G of Big Data Analytics Frameworks
Why apache Flink is the 4G of Big Data Analytics Frameworks
 
Design Patterns for Large-Scale Real-Time Learning
Design Patterns for Large-Scale Real-Time LearningDesign Patterns for Large-Scale Real-Time Learning
Design Patterns for Large-Scale Real-Time Learning
 
Spark Summit EU talk by Christos Erotocritou
Spark Summit EU talk by Christos ErotocritouSpark Summit EU talk by Christos Erotocritou
Spark Summit EU talk by Christos Erotocritou
 
Overview of Apache Fink: the 4 G of Big Data Analytics Frameworks
Overview of Apache Fink: the 4 G of Big Data Analytics FrameworksOverview of Apache Fink: the 4 G of Big Data Analytics Frameworks
Overview of Apache Fink: the 4 G of Big Data Analytics Frameworks
 
LinkedIn
LinkedInLinkedIn
LinkedIn
 
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
 
Rethinking Streaming Analytics For Scale
Rethinking Streaming Analytics For ScaleRethinking Streaming Analytics For Scale
Rethinking Streaming Analytics For Scale
 
Spark streaming
Spark streamingSpark streaming
Spark streaming
 
Streaming Analytics with Spark, Kafka, Cassandra and Akka
Streaming Analytics with Spark, Kafka, Cassandra and AkkaStreaming Analytics with Spark, Kafka, Cassandra and Akka
Streaming Analytics with Spark, Kafka, Cassandra and Akka
 
CERN’s Next Generation Data Analysis Platform with Apache Spark with Enric Te...
CERN’s Next Generation Data Analysis Platform with Apache Spark with Enric Te...CERN’s Next Generation Data Analysis Platform with Apache Spark with Enric Te...
CERN’s Next Generation Data Analysis Platform with Apache Spark with Enric Te...
 
Hadoop to spark_v2
Hadoop to spark_v2Hadoop to spark_v2
Hadoop to spark_v2
 
Apache Spark and the Emerging Technology Landscape for Big Data
Apache Spark and the Emerging Technology Landscape for Big DataApache Spark and the Emerging Technology Landscape for Big Data
Apache Spark and the Emerging Technology Landscape for Big Data
 
Apache Kafka Streams + Machine Learning / Deep Learning
Apache Kafka Streams + Machine Learning / Deep LearningApache Kafka Streams + Machine Learning / Deep Learning
Apache Kafka Streams + Machine Learning / Deep Learning
 
Flink vs. Spark
Flink vs. SparkFlink vs. Spark
Flink vs. Spark
 
Microservices, containers, and machine learning
Microservices, containers, and machine learningMicroservices, containers, and machine learning
Microservices, containers, and machine learning
 
Big Telco - Yousun Jeong
Big Telco - Yousun JeongBig Telco - Yousun Jeong
Big Telco - Yousun Jeong
 
Real-Time Anomoly Detection with Spark MLib, Akka and Cassandra by Natalino Busa
Real-Time Anomoly Detection with Spark MLib, Akka and Cassandra by Natalino BusaReal-Time Anomoly Detection with Spark MLib, Akka and Cassandra by Natalino Busa
Real-Time Anomoly Detection with Spark MLib, Akka and Cassandra by Natalino Busa
 

Viewers also liked

OSCON 2014: Data Workflows for Machine Learning
OSCON 2014: Data Workflows for Machine LearningOSCON 2014: Data Workflows for Machine Learning
OSCON 2014: Data Workflows for Machine Learning
Paco Nathan
 

Viewers also liked (20)

Strata 2015 Data Preview: Spark, Data Visualization, YARN, and More
Strata 2015 Data Preview: Spark, Data Visualization, YARN, and MoreStrata 2015 Data Preview: Spark, Data Visualization, YARN, and More
Strata 2015 Data Preview: Spark, Data Visualization, YARN, and More
 
Recipes for Running Spark Streaming Applications in Production-(Tathagata Das...
Recipes for Running Spark Streaming Applications in Production-(Tathagata Das...Recipes for Running Spark Streaming Applications in Production-(Tathagata Das...
Recipes for Running Spark Streaming Applications in Production-(Tathagata Das...
 
Lambda Architecture with Spark, Spark Streaming, Kafka, Cassandra, Akka and S...
Lambda Architecture with Spark, Spark Streaming, Kafka, Cassandra, Akka and S...Lambda Architecture with Spark, Spark Streaming, Kafka, Cassandra, Akka and S...
Lambda Architecture with Spark, Spark Streaming, Kafka, Cassandra, Akka and S...
 
Databricks Meetup @ Los Angeles Apache Spark User Group
Databricks Meetup @ Los Angeles Apache Spark User GroupDatabricks Meetup @ Los Angeles Apache Spark User Group
Databricks Meetup @ Los Angeles Apache Spark User Group
 
What's new with Apache Spark?
What's new with Apache Spark?What's new with Apache Spark?
What's new with Apache Spark?
 
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
 
Four Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and DatabricksFour Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and Databricks
 
Apache storm vs. Spark Streaming
Apache storm vs. Spark StreamingApache storm vs. Spark Streaming
Apache storm vs. Spark Streaming
 
Using Spark in Healthcare Predictive Analytics in the OR - Data Science Pop-u...
Using Spark in Healthcare Predictive Analytics in the OR - Data Science Pop-u...Using Spark in Healthcare Predictive Analytics in the OR - Data Science Pop-u...
Using Spark in Healthcare Predictive Analytics in the OR - Data Science Pop-u...
 
Application architectures with hadoop – big data techcon 2014
Application architectures with hadoop – big data techcon 2014Application architectures with hadoop – big data techcon 2014
Application architectures with hadoop – big data techcon 2014
 
Data Science in Future Tense
Data Science in Future TenseData Science in Future Tense
Data Science in Future Tense
 
#MesosCon 2014: Spark on Mesos
#MesosCon 2014: Spark on Mesos#MesosCon 2014: Spark on Mesos
#MesosCon 2014: Spark on Mesos
 
OSCON 2014: Data Workflows for Machine Learning
OSCON 2014: Data Workflows for Machine LearningOSCON 2014: Data Workflows for Machine Learning
OSCON 2014: Data Workflows for Machine Learning
 
How Apache Spark fits into the Big Data landscape
How Apache Spark fits into the Big Data landscapeHow Apache Spark fits into the Big Data landscape
How Apache Spark fits into the Big Data landscape
 
Big Data is changing abruptly, and where it is likely heading
Big Data is changing abruptly, and where it is likely headingBig Data is changing abruptly, and where it is likely heading
Big Data is changing abruptly, and where it is likely heading
 
Bring the Spark To Your Eyes
Bring the Spark To Your EyesBring the Spark To Your Eyes
Bring the Spark To Your Eyes
 
Future of data science as a profession
Future of data science as a professionFuture of data science as a profession
Future of data science as a profession
 
Big data & data science challenges and opportunities
Big data & data science   challenges and opportunitiesBig data & data science   challenges and opportunities
Big data & data science challenges and opportunities
 
Data Science in 2016: Moving Up
Data Science in 2016: Moving UpData Science in 2016: Moving Up
Data Science in 2016: Moving Up
 
Prediction as a service with ensemble model in SparkML and Python ScikitLearn
Prediction as a service with ensemble model in SparkML and Python ScikitLearnPrediction as a service with ensemble model in SparkML and Python ScikitLearn
Prediction as a service with ensemble model in SparkML and Python ScikitLearn
 

Similar to Strata EU 2014: Spark Streaming Case Studies

Media_Entertainment_Veriticals
Media_Entertainment_VeriticalsMedia_Entertainment_Veriticals
Media_Entertainment_Veriticals
Peyman Mohajerian
 
2024 Feb AI Meetup NYC GenAI_LLMs_ML_Data Codeless Generative AI Pipelines
2024 Feb AI Meetup NYC GenAI_LLMs_ML_Data Codeless Generative AI Pipelines2024 Feb AI Meetup NYC GenAI_LLMs_ML_Data Codeless Generative AI Pipelines
2024 Feb AI Meetup NYC GenAI_LLMs_ML_Data Codeless Generative AI Pipelines
Timothy Spann
 

Similar to Strata EU 2014: Spark Streaming Case Studies (20)

Overview of Apache Flink: the 4G of Big Data Analytics Frameworks
Overview of Apache Flink: the 4G of Big Data Analytics FrameworksOverview of Apache Flink: the 4G of Big Data Analytics Frameworks
Overview of Apache Flink: the 4G of Big Data Analytics Frameworks
 
Overview of Apache Fink: The 4G of Big Data Analytics Frameworks
Overview of Apache Fink: The 4G of Big Data Analytics FrameworksOverview of Apache Fink: The 4G of Big Data Analytics Frameworks
Overview of Apache Fink: The 4G of Big Data Analytics Frameworks
 
Media_Entertainment_Veriticals
Media_Entertainment_VeriticalsMedia_Entertainment_Veriticals
Media_Entertainment_Veriticals
 
Big data apache spark + scala
Big data   apache spark + scalaBig data   apache spark + scala
Big data apache spark + scala
 
Teaching Apache Spark: Demonstrations on the Databricks Cloud Platform
Teaching Apache Spark: Demonstrations on the Databricks Cloud PlatformTeaching Apache Spark: Demonstrations on the Databricks Cloud Platform
Teaching Apache Spark: Demonstrations on the Databricks Cloud Platform
 
Dev Ops Training
Dev Ops TrainingDev Ops Training
Dev Ops Training
 
End-to-End Data Pipelines with Apache Spark
End-to-End Data Pipelines with Apache SparkEnd-to-End Data Pipelines with Apache Spark
End-to-End Data Pipelines with Apache Spark
 
Spark Streaming the Industrial IoT
Spark Streaming the Industrial IoTSpark Streaming the Industrial IoT
Spark Streaming the Industrial IoT
 
Analysis-of-Major-Trends-in-big-data-analytics-slim-baltagi-hadoop-summit
Analysis-of-Major-Trends-in-big-data-analytics-slim-baltagi-hadoop-summitAnalysis-of-Major-Trends-in-big-data-analytics-slim-baltagi-hadoop-summit
Analysis-of-Major-Trends-in-big-data-analytics-slim-baltagi-hadoop-summit
 
Analysis of Major Trends in Big Data Analytics
Analysis of Major Trends in Big Data AnalyticsAnalysis of Major Trends in Big Data Analytics
Analysis of Major Trends in Big Data Analytics
 
Analysis of Major Trends in Big Data Analytics
Analysis of Major Trends in Big Data AnalyticsAnalysis of Major Trends in Big Data Analytics
Analysis of Major Trends in Big Data Analytics
 
Build Deep Learning Applications for Big Data Platforms (CVPR 2018 tutorial)
Build Deep Learning Applications for Big Data Platforms (CVPR 2018 tutorial)Build Deep Learning Applications for Big Data Platforms (CVPR 2018 tutorial)
Build Deep Learning Applications for Big Data Platforms (CVPR 2018 tutorial)
 
Scaling up with Cisco Big Data: Data + Science = Data Science
Scaling up with Cisco Big Data: Data + Science = Data ScienceScaling up with Cisco Big Data: Data + Science = Data Science
Scaling up with Cisco Big Data: Data + Science = Data Science
 
The Enterprise Guide to Building a Data Mesh - Introducing SpecMesh
The Enterprise Guide to Building a Data Mesh - Introducing SpecMeshThe Enterprise Guide to Building a Data Mesh - Introducing SpecMesh
The Enterprise Guide to Building a Data Mesh - Introducing SpecMesh
 
Huawei Advanced Data Science With Spark Streaming
Huawei Advanced Data Science With Spark StreamingHuawei Advanced Data Science With Spark Streaming
Huawei Advanced Data Science With Spark Streaming
 
IBM Strategy for Spark
IBM Strategy for SparkIBM Strategy for Spark
IBM Strategy for Spark
 
2024 Feb AI Meetup NYC GenAI_LLMs_ML_Data Codeless Generative AI Pipelines
2024 Feb AI Meetup NYC GenAI_LLMs_ML_Data Codeless Generative AI Pipelines2024 Feb AI Meetup NYC GenAI_LLMs_ML_Data Codeless Generative AI Pipelines
2024 Feb AI Meetup NYC GenAI_LLMs_ML_Data Codeless Generative AI Pipelines
 
Enterprise guide to building a Data Mesh
Enterprise guide to building a Data MeshEnterprise guide to building a Data Mesh
Enterprise guide to building a Data Mesh
 
Powering Data Science and AI with Apache Spark, Alluxio, and IBM
Powering Data Science and AI with Apache Spark, Alluxio, and IBMPowering Data Science and AI with Apache Spark, Alluxio, and IBM
Powering Data Science and AI with Apache Spark, Alluxio, and IBM
 
Open Source Lambda Architecture for deep learning
Open Source Lambda Architecture for deep learningOpen Source Lambda Architecture for deep learning
Open Source Lambda Architecture for deep learning
 

More from Paco Nathan

Human-in-the-loop: a design pattern for managing teams that leverage ML
Human-in-the-loop: a design pattern for managing teams that leverage MLHuman-in-the-loop: a design pattern for managing teams that leverage ML
Human-in-the-loop: a design pattern for managing teams that leverage ML
Paco Nathan
 
Human-in-a-loop: a design pattern for managing teams which leverage ML
Human-in-a-loop: a design pattern for managing teams which leverage MLHuman-in-a-loop: a design pattern for managing teams which leverage ML
Human-in-a-loop: a design pattern for managing teams which leverage ML
Paco Nathan
 
Humans in a loop: Jupyter notebooks as a front-end for AI
Humans in a loop: Jupyter notebooks as a front-end for AIHumans in a loop: Jupyter notebooks as a front-end for AI
Humans in a loop: Jupyter notebooks as a front-end for AI
Paco Nathan
 
Humans in the loop: AI in open source and industry
Humans in the loop: AI in open source and industryHumans in the loop: AI in open source and industry
Humans in the loop: AI in open source and industry
Paco Nathan
 
GraphX: Graph analytics for insights about developer communities
GraphX: Graph analytics for insights about developer communitiesGraphX: Graph analytics for insights about developer communities
GraphX: Graph analytics for insights about developer communities
Paco Nathan
 

More from Paco Nathan (18)

Human in the loop: a design pattern for managing teams working with ML
Human in the loop: a design pattern for managing  teams working with MLHuman in the loop: a design pattern for managing  teams working with ML
Human in the loop: a design pattern for managing teams working with ML
 
Human-in-the-loop: a design pattern for managing teams that leverage ML
Human-in-the-loop: a design pattern for managing teams that leverage MLHuman-in-the-loop: a design pattern for managing teams that leverage ML
Human-in-the-loop: a design pattern for managing teams that leverage ML
 
Human-in-a-loop: a design pattern for managing teams which leverage ML
Human-in-a-loop: a design pattern for managing teams which leverage MLHuman-in-a-loop: a design pattern for managing teams which leverage ML
Human-in-a-loop: a design pattern for managing teams which leverage ML
 
Humans in a loop: Jupyter notebooks as a front-end for AI
Humans in a loop: Jupyter notebooks as a front-end for AIHumans in a loop: Jupyter notebooks as a front-end for AI
Humans in a loop: Jupyter notebooks as a front-end for AI
 
Humans in the loop: AI in open source and industry
Humans in the loop: AI in open source and industryHumans in the loop: AI in open source and industry
Humans in the loop: AI in open source and industry
 
Computable Content
Computable ContentComputable Content
Computable Content
 
Computable Content: Lessons Learned
Computable Content: Lessons LearnedComputable Content: Lessons Learned
Computable Content: Lessons Learned
 
SF Python Meetup: TextRank in Python
SF Python Meetup: TextRank in PythonSF Python Meetup: TextRank in Python
SF Python Meetup: TextRank in Python
 
Use of standards and related issues in predictive analytics
Use of standards and related issues in predictive analyticsUse of standards and related issues in predictive analytics
Use of standards and related issues in predictive analytics
 
Data Science Reinvents Learning?
Data Science Reinvents Learning?Data Science Reinvents Learning?
Data Science Reinvents Learning?
 
Jupyter for Education: Beyond Gutenberg and Erasmus
Jupyter for Education: Beyond Gutenberg and ErasmusJupyter for Education: Beyond Gutenberg and Erasmus
Jupyter for Education: Beyond Gutenberg and Erasmus
 
GalvanizeU Seattle: Eleven Almost-Truisms About Data
GalvanizeU Seattle: Eleven Almost-Truisms About DataGalvanizeU Seattle: Eleven Almost-Truisms About Data
GalvanizeU Seattle: Eleven Almost-Truisms About Data
 
GraphX: Graph analytics for insights about developer communities
GraphX: Graph analytics for insights about developer communitiesGraphX: Graph analytics for insights about developer communities
GraphX: Graph analytics for insights about developer communities
 
Graph Analytics in Spark
Graph Analytics in SparkGraph Analytics in Spark
Graph Analytics in Spark
 
A New Year in Data Science: ML Unpaused
A New Year in Data Science: ML UnpausedA New Year in Data Science: ML Unpaused
A New Year in Data Science: ML Unpaused
 
How Apache Spark fits in the Big Data landscape
How Apache Spark fits in the Big Data landscapeHow Apache Spark fits in the Big Data landscape
How Apache Spark fits in the Big Data landscape
 
Brief Intro to Apache Spark @ Stanford ICME
Brief Intro to Apache Spark @ Stanford ICMEBrief Intro to Apache Spark @ Stanford ICME
Brief Intro to Apache Spark @ Stanford ICME
 
How Apache Spark fits into the Big Data landscape
How Apache Spark fits into the Big Data landscapeHow Apache Spark fits into the Big Data landscape
How Apache Spark fits into the Big Data landscape
 

Recently uploaded

+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
?#DUbAI#??##{{(☎️+971_581248768%)**%*]'#abortion pills for sale in dubai@
 
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers:  A Deep Dive into Serverless Spatial Data and FMECloud Frontiers:  A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
Safe Software
 

Recently uploaded (20)

"I see eyes in my soup": How Delivery Hero implemented the safety system for ...
"I see eyes in my soup": How Delivery Hero implemented the safety system for ..."I see eyes in my soup": How Delivery Hero implemented the safety system for ...
"I see eyes in my soup": How Delivery Hero implemented the safety system for ...
 
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemkeProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
 
Introduction to Multilingual Retrieval Augmented Generation (RAG)
Introduction to Multilingual Retrieval Augmented Generation (RAG)Introduction to Multilingual Retrieval Augmented Generation (RAG)
Introduction to Multilingual Retrieval Augmented Generation (RAG)
 
Vector Search -An Introduction in Oracle Database 23ai.pptx
Vector Search -An Introduction in Oracle Database 23ai.pptxVector Search -An Introduction in Oracle Database 23ai.pptx
Vector Search -An Introduction in Oracle Database 23ai.pptx
 
Six Myths about Ontologies: The Basics of Formal Ontology
Six Myths about Ontologies: The Basics of Formal OntologySix Myths about Ontologies: The Basics of Formal Ontology
Six Myths about Ontologies: The Basics of Formal Ontology
 
How to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerHow to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected Worker
 
+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
 
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
 
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers:  A Deep Dive into Serverless Spatial Data and FMECloud Frontiers:  A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
 
Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot Model
Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot ModelMcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot Model
Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot Model
 
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...Connector Corner: Accelerate revenue generation using UiPath API-centric busi...
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...
 
Apidays New York 2024 - Accelerating FinTech Innovation by Vasa Krishnan, Fin...
Apidays New York 2024 - Accelerating FinTech Innovation by Vasa Krishnan, Fin...Apidays New York 2024 - Accelerating FinTech Innovation by Vasa Krishnan, Fin...
Apidays New York 2024 - Accelerating FinTech Innovation by Vasa Krishnan, Fin...
 
FWD Group - Insurer Innovation Award 2024
FWD Group - Insurer Innovation Award 2024FWD Group - Insurer Innovation Award 2024
FWD Group - Insurer Innovation Award 2024
 
Navigating the Deluge_ Dubai Floods and the Resilience of Dubai International...
Navigating the Deluge_ Dubai Floods and the Resilience of Dubai International...Navigating the Deluge_ Dubai Floods and the Resilience of Dubai International...
Navigating the Deluge_ Dubai Floods and the Resilience of Dubai International...
 
[BuildWithAI] Introduction to Gemini.pdf
[BuildWithAI] Introduction to Gemini.pdf[BuildWithAI] Introduction to Gemini.pdf
[BuildWithAI] Introduction to Gemini.pdf
 
Web Form Automation for Bonterra Impact Management (fka Social Solutions Apri...
Web Form Automation for Bonterra Impact Management (fka Social Solutions Apri...Web Form Automation for Bonterra Impact Management (fka Social Solutions Apri...
Web Form Automation for Bonterra Impact Management (fka Social Solutions Apri...
 
MINDCTI Revenue Release Quarter One 2024
MINDCTI Revenue Release Quarter One 2024MINDCTI Revenue Release Quarter One 2024
MINDCTI Revenue Release Quarter One 2024
 
MS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectorsMS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectors
 
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
 
Corporate and higher education May webinar.pptx
Corporate and higher education May webinar.pptxCorporate and higher education May webinar.pptx
Corporate and higher education May webinar.pptx
 

Strata EU 2014: Spark Streaming Case Studies

  • 1. Spark Streaming Case Studies Strata EU, Barcelona 2014-11-20 strataconf.com/strataeu2014/public/schedule/detail/37493 Paco Nathan @pacoid
  • 3. Spark, the elevator pitch Developed in 2009 at UC Berkeley AMPLab, open sourced in 2010, Spark has since become one of the largest OSS communities in big data, with over 200 contributors in 50+ organizations spark.apache.org “Organizations that are looking at big data challenges – including collection, ETL, storage, exploration and analytics – should consider Spark for its in-memory performance and the breadth of its model. It supports advanced analytics solutions on Hadoop clusters, including the iterative model required for machine learning and graph analysis.” Gartner, Advanced Analytics and Data Science (2014) 3
  • 5. circa 2010: a unified engine for enterprise data workflows, based on commodity hardware a decade later… Spark: Cluster Computing with Working Sets Matei Zaharia, Mosharaf Chowdhury, Michael Franklin, Scott Shenker, Ion Stoica people.csail.mit.edu/matei/papers/2010/hotcloud_spark.pdf ! Resilient Distributed Datasets: A Fault-Tolerant Abstraction for In-Memory Cluster Computing Matei Zaharia, Mosharaf Chowdhury, Tathagata Das, Ankur Dave, Justin Ma, Murphy McCauley, Michael Franklin, Scott Shenker, Ion Stoica usenix.org/system/files/conference/nsdi12/nsdi12-final138.pdf 5 Spark, the elevator pitch
  • 6. Spark, the elevator pitch Spark Core is the general execution engine for the Spark platform that other functionality is built atop: ! • in-memory computing capabilities deliver speed • general execution model supports wide variety of use cases • ease of development – native APIs in Java, Scala, Python (+ SQL, Clojure, R) 6
  • 7. WordCount in 3 lines of Spark Spark, the elevator pitch WordCount in 50+ lines of Java MR 7
  • 8. Spark, the elevator pitch Sustained exponential growth, as one of the most active Apache projects ohloh.net/orgs/apache 8
  • 9. TL;DR: Smashing The Previous Petabyte Sort Record databricks.com/blog/2014/11/05/spark-officially-sets- a-new-record-in-large-scale-sorting.html 9
  • 11. Why Streaming? Because Machine Data! I <3 Logs Jay Kreps O’Reilly (2014) shop.oreilly.com/product/ 0636920034339.do 11
  • 12. Why Streaming? Because Google! MillWheel: Fault-Tolerant Stream Processing at Internet Scale Tyler Akidau, Alex Balikov, Kaya Bekiroglu, Slava Chernyak, Josh Haberman, Reuven Lax, Sam McVeety, Daniel Mills, Paul Nordstrom, Sam Whittle Very Large Data Bases (2013) research.google.com/pubs/ pub41378.html 12
  • 13. Why Streaming? Because IoT! kickstarter.com/projects/1614456084/b4rm4n-be- a-cocktail-hero 13
  • 14. Why Streaming? Because IoT! (exabytes/day per sensor) bits.blogs.nytimes.com/2013/06/19/g-e-makes-the-machine- and-then-uses-sensors-to-listen-to-it/ 14
  • 16. Spark Streaming: Requirements Let’s consider the top-level requirements for a streaming framework: • clusters scalable to 100’s of nodes • low-latency, in the range of seconds (meets 90% of use case needs) • efficient recovery from failures (which is a hard problem in CS) • integrates with batch: many co’s run the same business logic both online+offline 16
  • 17. Spark Streaming: Requirements Therefore, run a streaming computation as: a series of very small, deterministic batch jobs ! • 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 17
  • 18. Spark Streaming: Requirements Therefore, run a streaming computation as: a series of very small, deterministic batch jobs ! • Batch sizes as low as ½ sec, latency of about 1 sec • Potential for combining batch processing and streaming processing in the same system 18
  • 19. Spark Streaming: Integration Data can be ingested from many sources: Kafka, Flume, Twitter, ZeroMQ, TCP sockets, etc. Results can be pushed out to filesystems, databases, live dashboards, etc. Spark’s built-in machine learning algorithms and graph processing algorithms can be applied to data streams 19
  • 20. Spark Streaming: Timeline 2012 project started 2013 alpha release (Spark 0.7) 2014 graduated (Spark 0.9) Discretized Streams: A Fault-Tolerant Model for Scalable Stream Processing Matei Zaharia, Tathagata Das, Haoyuan Li, Timothy Hunter, Scott Shenker, Ion Stoica Berkeley EECS (2012-12-14) www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf project lead: Tathagata Das @tathadas 20
  • 21. Spark Streaming: Requirements Typical kinds of applications: • datacenter operations • web app funnel metrics • ad optimization • anti-fraud • various telematics and much much more! 21
  • 22. Spark Streaming: Some Excellent Resources Programming Guide spark.apache.org/docs/latest/streaming-programming- guide.html TD @ Spark Summit 2014 youtu.be/o-NXwFrNAWQ?list=PLTPXxbhUt- YWGNTaDj6HSjnHMxiTD1HCR “Deep Dive into Spark Streaming” slideshare.net/spark-project/deep-divewithsparkstreaming-tathagatadassparkmeetup20130617 Spark Reference Applications databricks.gitbooks.io/databricks-spark-reference- applications/ 22
  • 23. Quiz: name the bits and pieces… import org.apache.spark.streaming._! import org.apache.spark.streaming.StreamingContext._! ! // create a StreamingContext with a SparkConf configuration! val ssc = new StreamingContext(sparkConf, Seconds(10))! ! // create a DStream that will connect to serverIP:serverPort! val lines = ssc.socketTextStream(serverIP, serverPort)! ! // split each line into words! val words = lines.flatMap(_.split(" "))! ! // count each word in each batch! val pairs = words.map(word => (word, 1))! val wordCounts = pairs.reduceByKey(_ + _)! ! // print a few of the counts to the console! wordCounts.print()! ! ssc.start()! ssc.awaitTermination() 23
  • 25. Because Use Cases: +40 known production use cases 25
  • 26. Because Use Cases: Analysis Reasons for adopting/transitioning to Spark Streaming… the unified programming model is particularly relevant for real-time analytics that combine historical data: • Making data science accessible to non-scientists • Higher productivity for data workers • Exactly-once semantics • No compromises on scalability and throughput • Ease of operations 26
  • 27. Because Use Cases: Stratio Stratio Streaming: a new approach to Spark Streaming David Morales, Oscar Mendez 2014-06-30 spark-summit.org/2014/talk/stratio-streaming- a-new-approach-to-spark-streaming • Stratio Streaming is the union of a real-time messaging bus with a complex event processing engine using Spark Streaming • allows the creation of streams and queries on the fly • paired with Siddhi CEP engine and Apache Kafka • added global features to the engine such as auditing 27 and statistics
  • 28. Because Use Cases: Ooyala Productionizing a 24/7 Spark Streaming service on YARN Issac Buenrostro, Arup Malakar 2014-06-30 spark-summit.org/2014/talk/ productionizing-a-247-spark-streaming-service- on-yarn • state-of-the-art ingestion pipeline, processing over two billion video events a day • how do you ensure 24/7 availability and fault tolerance? • what are the best practices for Spark Streaming and its integration with Kafka and YARN? • how do you monitor and instrument the various 28 stages of the pipeline?
  • 29. Because Use Cases: Guavus Guavus Embeds Apache Spark into its Operational Intelligence Platform Deployed at the World’s Largest Telcos Eric Carr 2014-09-25 databricks.com/blog/2014/09/25/guavus-embeds-apache-spark-into- its-operational-intelligence-platform-deployed-at-the-worlds- largest-telcos.html • 4 of 5 top mobile network operators, 3 of 5 top Internet backbone providers, 80% MSOs in NorAm • analyzing 50% of US mobile data traffic, +2.5 PB/day • latency is critical for resolving operational issues before they cascade: 2.5 MM transactions per second • “analyze first” not “store first ask questions later” 29
  • 30. Because Use Cases: Sharethrough Sharethrough Uses Spark Streaming to Optimize Bidding in Real Time Russell Cardullo, Michael Ruggier 2014-03-25 databricks.com/blog/2014/03/25/ sharethrough-and-spark-streaming.html • the profile of a 24 x 7 streaming app is different than an hourly batch job… • take time to validate output against the input… • confirm that supporting objects are being serialized… • the output of your Spark Streaming job is only as reliable as the queue that feeds Spark… • monoids… 30
  • 31. Because Use Cases: Viadeo Spark Streaming As Near Realtime ETL Djamel Zouaoui 2014-09-18 slideshare.net/DjamelZouaoui/spark-streaming • Spark Streaming is topology-free • workers and receivers are autonomous and independent • paired with Kafka, RabbitMQ • 8 machines / 120 cores • use case for recommender system • issues: how to handle lost data, serialization 31
  • 33. Demos: brand new Python support for Streaming in 1.2 github.com/apache/spark/tree/master/examples/src/main/ python/streaming ! For more Spark learning resources online: databricks.com/spark-training-resources 33
  • 34. Demo: PySpark Streaming Network Word Count import sys! from pyspark import SparkContext! from pyspark.streaming import StreamingContext! ! sc = SparkContext(appName="PyStreamNWC", master="local[*]")! ssc = StreamingContext(sc, Seconds(5))! ! lines = ssc.socketTextStream(sys.argv[1], int(sys.argv[2]))! ! counts = lines.flatMap(lambda line: line.split(" ")) ! .map(lambda word: (word, 1)) ! .reduceByKey(lambda a, b: a+b)! ! counts.pprint()! ! ssc.start()! ssc.awaitTermination() 34
  • 35. Demo: PySpark Streaming Network Word Count - Stateful import sys! from pyspark import SparkContext! from pyspark.streaming import StreamingContext! ! def updateFunc (new_values, last_sum):! return sum(new_values) + (last_sum or 0)! ! sc = SparkContext(appName="PyStreamNWC", master="local[*]")! ssc = StreamingContext(sc, Seconds(5))! ssc.checkpoint("checkpoint")! ! lines = ssc.socketTextStream(sys.argv[1], int(sys.argv[2]))! ! counts = lines.flatMap(lambda line: line.split(" ")) ! .map(lambda word: (word, 1)) ! .updateStateByKey(updateFunc) ! .transform(lambda x: x.sortByKey())! ! counts.pprint()! ! ssc.start()! ssc.awaitTermination() 35
  • 37. Spark Integrations: Discover Insights Clean Up Your Data Run Sophisticated Analytics Integrate With Many Other Systems Use Lots of Different Data Sources cloud-based notebooks… ETL… the Hadoop ecosystem… widespread use of PyData… advanced analytics in streaming… rich custom search… web apps for data APIs… low-latency + multi-tenancy… 37
  • 38. Spark Integrations: Advanced analytics for streaming use cases Kafka + Spark + Cassandra datastax.com/documentation/datastax_enterprise/4.5/ datastax_enterprise/spark/sparkIntro.html http://helenaedelson.com/?p=991 github.com/datastax/spark-cassandra-connector github.com/dibbhatt/kafka-spark-consumer unified compute data streams columnar key-value 38
  • 39. Spark Integrations: Rich search, immediate insights Spark + ElasticSearch databricks.com/blog/2014/06/27/application-spotlight-elasticsearch. html elasticsearch.org/guide/en/elasticsearch/hadoop/current/ spark.html spark-summit.org/2014/talk/streamlining-search-indexing- using-elastic-search-and-spark unified compute document search 39
  • 40. Spark Integrations: General Guidelines • use Tachyon as a best practice for sharing between two streaming apps • or write to Cassandra or HBase / then read back • design patterns for integration: spark.apache.org/docs/latest/streaming-programming- guide.html#output-operations- on-dstreams 40
  • 42. A Look Ahead… 1. Greater Stability and Robustness • improved high availability via write-ahead logs • enabled as an optional feature for Spark 1.2 • NB: Spark Standalone can already restart driver • excellent discussion of fault-tolerance (2012): cs.duke.edu/~kmoses/cps516/dstream.html • stay tuned: meetup.com/spark-users/events/218108702/ 42
  • 43. A Look Ahead… 2. Support for more environments, i.e., beyond Hadoop • three use cases currently depend on HDFS • those are being abstracted out • could then use Cassandra, etc. 43
  • 44. A Look Ahead… 3. Improved support for Python • e.g., Kafka is not exposed through Python yet (next release goal) 44
  • 45. A Look Ahead… 4. Better flow control • a somewhat longer-term goal, plus it is a hard problem in general • poses interesting challenges beyond what other streaming systems have faced 45
  • 47. A Big Picture… 19-20c. statistics emphasized defensibility in lieu of predictability, based on analytic variance and goodness-of-fit tests ! That approach inherently led toward a manner of computational thinking based on batch windows ! They missed a subtle point… 47
  • 48. A Big Picture… The view in the lens has changed 21c. shift towards modeling based on probabilistic approximations: trade bounded errors for greatly reduced resource costs highlyscalable.wordpress.com/2012/05/01/ probabilistic-structures-web-analytics-data- mining/ 48
  • 49. A Big Picture… The view in the lens has changed 21c. shift towards modeling based on probabil approximations: trade bounded errors for greatly reduced resource costs Twitter catch-phrase: “Hash, don’t sample” highlyscalable.wordpress.com/2012/05/01/ probabilistic-structures-web-analytics-data- mining/ 49
  • 50. Probabilistic Data Structures: a fascinating and relatively new area, pioneered by relatively few people – e.g., Philippe Flajolet provides approximation, with error bounds – in general uses significantly less resources (RAM, CPU, etc.) many algorithms can be constructed from combinations of read and write monoids aggregate different ranges by composing hashes, instead of repeating full-queries 50
  • 51. Probabilistic Data Structures: Some Examples algorithm use case example Count-Min Sketch frequency summaries code HyperLogLog set cardinality code Bloom Filter set membership MinHash set similarity DSQ streaming quantiles SkipList ordered sequence search 51
  • 52. Probabilistic Data Structures: Some Examples algorithm use case example Count-Min Sketch frequency summaries code HyperLogLog set cardinality code suggestion: consider these as your most quintessential collections data types at scale Bloom Filter set membership MinHash set similarity DSQ streaming quantiles SkipList ordered sequence search 52
  • 53. Add ALL the Things: Abstract Algebra Meets Analytics infoq.com/presentations/abstract-algebra-analytics Avi Bryant, Strange Loop (2013) • grouping doesn’t matter (associativity) • ordering doesn’t matter (commutativity) • zeros get ignored In other words, while partitioning data at scale is quite difficult, you can let the math allow your code to be flexible at scale Avi Bryant @avibryant Probabilistic Data Structures: Performance Bottlenecks 53
  • 54. Probabilistic Data Structures: Industry Drivers • sketch algorithms: trade bounded errors for orders of magnitude less required resources, e.g., fit more complex apps in memory • multicore + large memory spaces (off heap) are increasing the resources per node in a cluster • containers allow for finer-grain allocation of cluster resources and multi-tenancy • monoids, etc.: guarantees of associativity within the code allow for more effective distributed computing, e.g., partial aggregates • less resources must be spent sorting/windowing data prior to working with a data set • real-time apps, which don’t have the luxury of anticipating data partitions, can respond quickly 54
  • 55. Probabilistic Data Structures: Recommended Reading Probabilistic Data Structures for Web Analytics and Data Mining Ilya Katsov (2012-05-01) A collection of links for streaming algorithms and data structures Debasish Ghosh Aggregate Knowledge blog (now Neustar) Timon Karnezos, Matt Curcio, et al. Probabilistic Data Structures and Breaking Down Big Sequence Data C. Titus Brown, O'Reilly (2010-11-10) Algebird Avi Bryant, Oscar Boykin, et al. Twitter (2012) Mining of Massive Datasets Jure Leskovec, Anand Rajaraman, Jeff Ullman, Cambridge (2011) 55
  • 57. cloud-based notebooks: databricks.com/blog/2014/07/14/databricks-cloud-making- big-data-easy.html youtube.com/watch?v=dJQ5lV5Tldw#t=883 57
  • 58. certification: Apache Spark developer certificate program • http://oreilly.com/go/sparkcert • defined by Spark experts @Databricks • assessed by O’Reilly Media • establishes the bar for Spark expertise 58
  • 59. community: spark.apache.org/community.html video+slide archives: spark-summit.org events worldwide: goo.gl/2YqJZK resources: databricks.com/spark-training-resources workshops: databricks.com/spark-training 59
  • 60. books: Fast Data Processing with Spark Holden Karau Packt (2013) shop.oreilly.com/product/ 9781782167068.do Spark in Action Chris Fregly Manning (2015*) sparkinaction.com/ Learning Spark Holden Karau, Andy Konwinski, Matei Zaharia O’Reilly (2015*) shop.oreilly.com/product/ 0636920028512.do 60
  • 61. events: Strata EU Barcelona, Nov 19-21 strataconf.com/strataeu2014 Data Day Texas Austin, Jan 10 datadaytexas.com Strata CA San Jose, Feb 18-20 strataconf.com/strata2015 Spark Summit East NYC, Mar 18-19 spark-summit.org/east Strata EU London, May 5-7 strataconf.com/big-data-conference-uk-2015 Spark Summit 2015 SF, Jun 15-17 spark-summit.org 61
  • 62. presenter: monthly newsletter for updates, events, conf summaries, etc.: liber118.com/pxn/ Just Enough Math O’Reilly, 2014 justenoughmath.com preview: youtu.be/TQ58cWgdCpA Enterprise Data Workflows with Cascading O’Reilly, 2013 shop.oreilly.com/product/ 0636920028536.do 62