SlideShare a Scribd company logo
1 of 45
Spark Streaming
Spark streaming recipes and “exactly once” semantics revised
Appsflyer: Basic Flow
Advertiser
Publisher
Click
Install
Appsflyer as Marketing Platform
Attribution
Statistics: clicks, installs, in-app events, launches, uninstalls, etc.
Life time value
Retargeting
Fraud detection
Prediction
A/B testing
etc...
Appsflyer Technology
~7B events / day
Hundreds of machines in Amazon
Tens of micro-services
Apache Kafka
service
service
service
service
service
service
DB
Amazon S3
MongoDB
Redshift
Druid
What is stream processing?
Stream Processing
Minimize latency between data ingestion and insights
Usages
● Real-time dashboard
● Fraud prevention
● Ad bidding
● etc.
Stream Processing Frameworks
Key Differences
● Latency
● Windowing support
● Delivery semantics
● State management
● API easiness
● Programming languages support
● Community support
● etc..
Apache Spark
Spark Driver
val textFile = sc.textFile("hdfs://...")
val counts = textFile
.flatMap(line => line.split(" "))
.map(word => (word, 1))
.reduceByKey(_ + _)
counts.saveAsTextFile("hdfs://...")
Cluster Manager
Worker Node
Executor
Task Task
Worker Node
Executor
Task Task
Apache Spark
External World
RDD RDD
RDD RDD
RDD External World
read
read
transform
transform
transform
transform
action
Streaming in Spark
Advantages
● Reuse existing infra
● Rich API
● Straightforward windowing
● It’s easier to implement “exactly once”
Disadvantages
● Latency
Input Stream Micro batches
Spark Engine
Processed data
Windowing in Spark Streaming
● Window length and sliding interval must be multiples of batch interval
● Possible usages
○ Finding Top N elements during last M period of time
○ Pre-aggregation of data prior to inserting to DB
○ etc.
DStream
Window length
Sliding interval
Do we need “exactly once” semantics?
Data Processing Paradigms
Batch layer
Real-time layer
Real-time layer
http://www.kappa-architecture.com/https://en.wikipedia.org/wiki/Lambda_architecture
How do we achieve “exactly once”?
Achieving “Exactly once”
Producer
Doesn’t duplicate messages
Stream processor
Tracking state (checkpointing)
Resilient components
Consumer
Reads only new messages
“Easy” way:
Message deduplication based on some ID
Idempotent output destination
Stream Checkpointing
https://en.wikipedia.org/wiki/Snapshot_algorithm
Barriers are injected into the data stream.
Once intermediate step sees barriers from all of its input streams it outputs barrier to all of its outgoing streams.
Once all sink operators see barrier for a snapshot, they acknowledge the snapshot, and it’s considered committed.
Multiple barriers can be seen in stream flow.
Operators store their state to an external storage.
storage
Micro-batch Checkpointing
receive
process
state
receive
process
state
receive
process
state
while (true) {
// 1. receive next batch of data
// 2. compute next stream and state
}
Unit of fault tolerance
Resilience in Spark Streaming
All Spark components must be resilient!
Driver application process
Master process
Worker process
Executor process
Receiver thread
Worker node
Driver
Master
Worker Node
Executor
Tas
k
Tas
k
Worker Node
Executor
Tas
k
Tas
k
Driver
Driver Resilience
Client mode
Driver application is running inside the “spark-submit” process.
If this process dies the entire application is killed.
Cluster mode
Driver application runs on one of worker nodes.
“--supervise” option makes driver restart
on a different worker node.
Running through Marathon
Marathon can re-start failed
applications automatically.
Master
Worker Node
Executor
Tas
k
Tas
k
Worker Node
Executor
Tas
k
Tas
k
Master Resilience
Single master
The entire application is killed.
Multi-master mode
A standby master is elected active.
Worker nodes automatically register with new master.
Leader election via ZooKeeper. Driver
Master
Worker Node
Executor
Tas
k
Tas
k
Worker Node
Executor
Tas
k
Tas
k
Worker Resilience
Worker process
When failed, all child processes (driver or executor) are killed.
New worker process is launched automatically.
Executor process
Restarted on failure by the parent worker process.
Receiver thread
Running inside the Executor process -
same as Executor.
Worker node
Driver
Master
Worker Node
Executor
Tas
k
Tas
k
Worker Node
Executor
Tas
k
Tas
k
Resilience doesn’t ensure “exactly once”
Checkpointing
Checkpointing helps recover from driver failure.
Stores computation graph to some fault tolerant place (like HDFS or S3).
What is saved as metadata
Metadata of queued but not processed batches
Stream operations (code)
Configuration
Disadvantages
Frequent checkpointing reduces throughput.
Write Ahead Log
Synchronously saves received data to fault tolerant storage.
Helps recover received, but not yet committed blocks.
Disadvantages
Additional storage is required.
Reduced throughput.
Executor
input stream
Receiver
Problems with Checkpointing and WAL
Data can be lost even when using checkpointing (batches hold in memory will be
lost on driver failure).
Checkpointing and WAL prevent data loss, but do not provide “exactly once”
semantics.
If receiver fails before updating offsets in ZooKeeper - we are in trouble.
In this case data will be re-read from Kafka and from WAL.
Still not exactly once!
The Solution
Don’t use receivers - read directly from input stream instead.
Driver instructs executors what range to read from a stream (stream must be rewindable).
Read range is attached to the batch itself.
Example (Kafka direct stream):
Application Driver
Streaming
Context
1. Periodically query latest
offsets for topics & partitions
2. Calculates offset ranges for
the next batch
Executor
3. Schedule the next micro-
batch job
4. Consume data for the
calculated offsets
Example #1
The Problem
Events counting.
Group by different set of dimensions.
Have pre-aggregation layer that reduces load on DB on spikes.
DB
app_id event_name country count
com.app.bla FIRST_LAUNCH US 152
com.app.bla purchase IL 10
com.app.jo custom_inapp_20 US 45
Transactional Events Aggregator
Based on SQL database
Store Kafka partition offsets into the DB
Increment event counters in transaction based on current and stored offsets.
SQL DB
Driver Executor Executor
1. Read last Kafka partitions and their
offsets from the DB
2. Create direct Kafka stream
based on read partitions and
offsets
3. Consume events from Kafka
4. Aggregate events
5. Upsert event counter along with
current offsets in transaction
Creating Kafka Stream
Aggregation & Writing to DB
Example #2
Snapshotting Events Aggregator
Driver Executor Executor
1. Read last Kafka partitions
and their offsets from S3
2. Create direct Kafka stream
based on read partitions and
offsets
3. Consume events from Kafka
4. Aggregate events
5. Store processed data and Kafka offsets
under /data/ts=<timestamp> and
/offsets/ts=<timestamp> respectively
S3
Aggregator Application
Snapshotting Events Aggregator
Executor
Executor Executor
1. Find last committed timestamp
2. Read data for the last
timestamp from
/data/ts=<timestamp>
4. Aggregate events by
different dimensions, and split
to cubes
6. Delete offsets and data for the timestamp
/offsets/ts=<timestamp>
/data/ts=<timestamp>
S3
Loader Application
Cassandra
5. Increment counters in
different cubes
Driver
Aggregator
Aggregator
Loader
Deployment
We use Mesos
Master HA for free.
Marathon keeps Spark streaming application alive.
Read carefully
http://spark.apache.org/docs/latest/streaming-programming-guide.html#performance-tuning
Inspect, re-configure, retry
Turn off Spark dynamicity
Preserve data locality
Find balance between cores/batch interval/block interval
Processing time must be less than batch interval
Tips
Thank you!
(and we’re hiring)
Right Now
Real time analytics dashboard
Right Now
Processes ~50M events a day
Reduces the stream in two sliding windows:
1. Last 5 seconds (“now”)
2. Last 10 minutes (“recent”)
At most once semantics
Right Now
Right Now
Why Spark?
Experienced with Spark
Convenient Clojure wrappers (Sparkling, Flambo)
Documentation and community
Right Now
In Production
3 m3.xlarge machines for the workers (4 cores each)
spark.default.parallelism=10
Lesson learned: foreachRDD and foreachPartition
Thank you!

More Related Content

What's hot

Being Ready for Apache Kafka - Apache: Big Data Europe 2015
Being Ready for Apache Kafka - Apache: Big Data Europe 2015Being Ready for Apache Kafka - Apache: Big Data Europe 2015
Being Ready for Apache Kafka - Apache: Big Data Europe 2015Michael Noll
 
Building Stateful Microservices With Akka
Building Stateful Microservices With AkkaBuilding Stateful Microservices With Akka
Building Stateful Microservices With AkkaYaroslav Tkachenko
 
Spark streaming and Kafka
Spark streaming and KafkaSpark streaming and Kafka
Spark streaming and KafkaIraj Hedayati
 
Functional Comparison and Performance Evaluation of Streaming Frameworks
Functional Comparison and Performance Evaluation of Streaming FrameworksFunctional Comparison and Performance Evaluation of Streaming Frameworks
Functional Comparison and Performance Evaluation of Streaming FrameworksHuafeng Wang
 
Arbitrary Stateful Aggregations using Structured Streaming in Apache Spark
Arbitrary Stateful Aggregations using Structured Streaming in Apache SparkArbitrary Stateful Aggregations using Structured Streaming in Apache Spark
Arbitrary Stateful Aggregations using Structured Streaming in Apache SparkDatabricks
 
Real Time Data Streaming using Kafka & Storm
Real Time Data Streaming using Kafka & StormReal Time Data Streaming using Kafka & Storm
Real Time Data Streaming using Kafka & StormRan Silberman
 
Real-time streaming and data pipelines with Apache Kafka
Real-time streaming and data pipelines with Apache KafkaReal-time streaming and data pipelines with Apache Kafka
Real-time streaming and data pipelines with Apache KafkaJoe Stein
 
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
 
Building Scalable and Extendable Data Pipeline for Call of Duty Games: Lesson...
Building Scalable and Extendable Data Pipeline for Call of Duty Games: Lesson...Building Scalable and Extendable Data Pipeline for Call of Duty Games: Lesson...
Building Scalable and Extendable Data Pipeline for Call of Duty Games: Lesson...Yaroslav Tkachenko
 
Building Continuous Application with Structured Streaming and Real-Time Data ...
Building Continuous Application with Structured Streaming and Real-Time Data ...Building Continuous Application with Structured Streaming and Real-Time Data ...
Building Continuous Application with Structured Streaming and Real-Time Data ...Databricks
 
Apache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARN
Apache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARNApache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARN
Apache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARNblueboxtraveler
 
Achieving a 50% Reduction in Cross-AZ Network Costs from Kafka (Uday Sagar Si...
Achieving a 50% Reduction in Cross-AZ Network Costs from Kafka (Uday Sagar Si...Achieving a 50% Reduction in Cross-AZ Network Costs from Kafka (Uday Sagar Si...
Achieving a 50% Reduction in Cross-AZ Network Costs from Kafka (Uday Sagar Si...confluent
 
Introduction to Kafka Streams
Introduction to Kafka StreamsIntroduction to Kafka Streams
Introduction to Kafka StreamsGuozhang Wang
 
Deep dive into stateful stream processing in structured streaming by Tathaga...
Deep dive into stateful stream processing in structured streaming  by Tathaga...Deep dive into stateful stream processing in structured streaming  by Tathaga...
Deep dive into stateful stream processing in structured streaming by Tathaga...Databricks
 
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...confluent
 
Real Time Streaming Data with Kafka and TensorFlow (Yong Tang, MobileIron) Ka...
Real Time Streaming Data with Kafka and TensorFlow (Yong Tang, MobileIron) Ka...Real Time Streaming Data with Kafka and TensorFlow (Yong Tang, MobileIron) Ka...
Real Time Streaming Data with Kafka and TensorFlow (Yong Tang, MobileIron) Ka...confluent
 

What's hot (20)

Stream Processing made simple with Kafka
Stream Processing made simple with KafkaStream Processing made simple with Kafka
Stream Processing made simple with Kafka
 
Being Ready for Apache Kafka - Apache: Big Data Europe 2015
Being Ready for Apache Kafka - Apache: Big Data Europe 2015Being Ready for Apache Kafka - Apache: Big Data Europe 2015
Being Ready for Apache Kafka - Apache: Big Data Europe 2015
 
Building Stateful Microservices With Akka
Building Stateful Microservices With AkkaBuilding Stateful Microservices With Akka
Building Stateful Microservices With Akka
 
Spark streaming and Kafka
Spark streaming and KafkaSpark streaming and Kafka
Spark streaming and Kafka
 
Functional Comparison and Performance Evaluation of Streaming Frameworks
Functional Comparison and Performance Evaluation of Streaming FrameworksFunctional Comparison and Performance Evaluation of Streaming Frameworks
Functional Comparison and Performance Evaluation of Streaming Frameworks
 
Arbitrary Stateful Aggregations using Structured Streaming in Apache Spark
Arbitrary Stateful Aggregations using Structured Streaming in Apache SparkArbitrary Stateful Aggregations using Structured Streaming in Apache Spark
Arbitrary Stateful Aggregations using Structured Streaming in Apache Spark
 
The Stream Processor as a Database Apache Flink
The Stream Processor as a Database Apache FlinkThe Stream Processor as a Database Apache Flink
The Stream Processor as a Database Apache Flink
 
Spark streaming + kafka 0.10
Spark streaming + kafka 0.10Spark streaming + kafka 0.10
Spark streaming + kafka 0.10
 
Real Time Data Streaming using Kafka & Storm
Real Time Data Streaming using Kafka & StormReal Time Data Streaming using Kafka & Storm
Real Time Data Streaming using Kafka & Storm
 
Real-time streaming and data pipelines with Apache Kafka
Real-time streaming and data pipelines with Apache KafkaReal-time streaming and data pipelines with Apache Kafka
Real-time streaming and data pipelines with Apache Kafka
 
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
 
Building Scalable and Extendable Data Pipeline for Call of Duty Games: Lesson...
Building Scalable and Extendable Data Pipeline for Call of Duty Games: Lesson...Building Scalable and Extendable Data Pipeline for Call of Duty Games: Lesson...
Building Scalable and Extendable Data Pipeline for Call of Duty Games: Lesson...
 
Building Continuous Application with Structured Streaming and Real-Time Data ...
Building Continuous Application with Structured Streaming and Real-Time Data ...Building Continuous Application with Structured Streaming and Real-Time Data ...
Building Continuous Application with Structured Streaming and Real-Time Data ...
 
Apache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARN
Apache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARNApache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARN
Apache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARN
 
Achieving a 50% Reduction in Cross-AZ Network Costs from Kafka (Uday Sagar Si...
Achieving a 50% Reduction in Cross-AZ Network Costs from Kafka (Uday Sagar Si...Achieving a 50% Reduction in Cross-AZ Network Costs from Kafka (Uday Sagar Si...
Achieving a 50% Reduction in Cross-AZ Network Costs from Kafka (Uday Sagar Si...
 
Introduction to Kafka Streams
Introduction to Kafka StreamsIntroduction to Kafka Streams
Introduction to Kafka Streams
 
ApacheCon BigData Europe 2015
ApacheCon BigData Europe 2015 ApacheCon BigData Europe 2015
ApacheCon BigData Europe 2015
 
Deep dive into stateful stream processing in structured streaming by Tathaga...
Deep dive into stateful stream processing in structured streaming  by Tathaga...Deep dive into stateful stream processing in structured streaming  by Tathaga...
Deep dive into stateful stream processing in structured streaming by Tathaga...
 
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...
Kafka Summit NYC 2017 - Easy, Scalable, Fault-tolerant Stream Processing with...
 
Real Time Streaming Data with Kafka and TensorFlow (Yong Tang, MobileIron) Ka...
Real Time Streaming Data with Kafka and TensorFlow (Yong Tang, MobileIron) Ka...Real Time Streaming Data with Kafka and TensorFlow (Yong Tang, MobileIron) Ka...
Real Time Streaming Data with Kafka and TensorFlow (Yong Tang, MobileIron) Ka...
 

Viewers also liked

Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)
Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)
Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)Spark Summit
 
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das Databricks
 
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...Spark Summit
 
INTRODUCING: CREATE PIPELINE
INTRODUCING: CREATE PIPELINEINTRODUCING: CREATE PIPELINE
INTRODUCING: CREATE PIPELINESingleStore
 
Spark, Deep Learning and Life Sciences, Systems Biology in the Big Data Age
Spark, Deep Learning and Life Sciences, Systems Biology in the Big Data AgeSpark, Deep Learning and Life Sciences, Systems Biology in the Big Data Age
Spark, Deep Learning and Life Sciences, Systems Biology in the Big Data Agebatchinsights
 
Partner Development Guide for Kafka Connect
Partner Development Guide for Kafka ConnectPartner Development Guide for Kafka Connect
Partner Development Guide for Kafka Connectconfluent
 
Streaming Analytics & CEP - Two sides of the same coin?
Streaming Analytics & CEP - Two sides of the same coin?Streaming Analytics & CEP - Two sides of the same coin?
Streaming Analytics & CEP - Two sides of the same coin?Till Rohrmann
 
Oracle GoldenGate and Apache Kafka A Deep Dive Into Real-Time Data Streaming
Oracle GoldenGate and Apache Kafka A Deep Dive Into Real-Time Data StreamingOracle GoldenGate and Apache Kafka A Deep Dive Into Real-Time Data Streaming
Oracle GoldenGate and Apache Kafka A Deep Dive Into Real-Time Data StreamingMichael Rainey
 
Spark streaming with kafka
Spark streaming with kafkaSpark streaming with kafka
Spark streaming with kafkaDori Waldman
 
Meet Up - Spark Stream Processing + Kafka
Meet Up - Spark Stream Processing + KafkaMeet Up - Spark Stream Processing + Kafka
Meet Up - Spark Stream Processing + KafkaKnoldus Inc.
 
No data loss pipeline with apache kafka
No data loss pipeline with apache kafkaNo data loss pipeline with apache kafka
No data loss pipeline with apache kafkaJiangjie Qin
 
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...Databricks
 
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17spark-project
 
Top 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsTop 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsSpark Summit
 
Four Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and DatabricksFour Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and DatabricksLegacy Typesafe (now Lightbend)
 
Why your Spark Job is Failing
Why your Spark Job is FailingWhy your Spark Job is Failing
Why your Spark Job is FailingDataWorks Summit
 

Viewers also liked (20)

Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)
Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)
Exactly-Once Streaming from Kafka-(Cody Koeninger, Kixer)
 
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das Apache Spark 2.0: A Deep Dive Into Structured Streaming - by Tathagata Das
Apache Spark 2.0: A Deep Dive Into Structured Streaming - by 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...
Recipes for Running Spark Streaming Applications in Production-(Tathagata Das...
 
Venkatasiva reddy lekkala
Venkatasiva reddy lekkala Venkatasiva reddy lekkala
Venkatasiva reddy lekkala
 
INTRODUCING: CREATE PIPELINE
INTRODUCING: CREATE PIPELINEINTRODUCING: CREATE PIPELINE
INTRODUCING: CREATE PIPELINE
 
Spark, Deep Learning and Life Sciences, Systems Biology in the Big Data Age
Spark, Deep Learning and Life Sciences, Systems Biology in the Big Data AgeSpark, Deep Learning and Life Sciences, Systems Biology in the Big Data Age
Spark, Deep Learning and Life Sciences, Systems Biology in the Big Data Age
 
Partner Development Guide for Kafka Connect
Partner Development Guide for Kafka ConnectPartner Development Guide for Kafka Connect
Partner Development Guide for Kafka Connect
 
Streaming Analytics & CEP - Two sides of the same coin?
Streaming Analytics & CEP - Two sides of the same coin?Streaming Analytics & CEP - Two sides of the same coin?
Streaming Analytics & CEP - Two sides of the same coin?
 
Oracle GoldenGate and Apache Kafka A Deep Dive Into Real-Time Data Streaming
Oracle GoldenGate and Apache Kafka A Deep Dive Into Real-Time Data StreamingOracle GoldenGate and Apache Kafka A Deep Dive Into Real-Time Data Streaming
Oracle GoldenGate and Apache Kafka A Deep Dive Into Real-Time Data Streaming
 
Spark streaming with kafka
Spark streaming with kafkaSpark streaming with kafka
Spark streaming with kafka
 
Dataflow with Apache NiFi - Crash Course - HS16SJ
Dataflow with Apache NiFi - Crash Course - HS16SJDataflow with Apache NiFi - Crash Course - HS16SJ
Dataflow with Apache NiFi - Crash Course - HS16SJ
 
Apache Kafka Security
Apache Kafka Security Apache Kafka Security
Apache Kafka Security
 
Meet Up - Spark Stream Processing + Kafka
Meet Up - Spark Stream Processing + KafkaMeet Up - Spark Stream Processing + Kafka
Meet Up - Spark Stream Processing + Kafka
 
No data loss pipeline with apache kafka
No data loss pipeline with apache kafkaNo data loss pipeline with apache kafka
No data loss pipeline with apache kafka
 
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
Structuring Apache Spark 2.0: SQL, DataFrames, Datasets And Streaming - by Mi...
 
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
 
Top 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsTop 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark Applications
 
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
 
Why your Spark Job is Failing
Why your Spark Job is FailingWhy your Spark Job is Failing
Why your Spark Job is Failing
 
Yahoo compares Storm and Spark
Yahoo compares Storm and SparkYahoo compares Storm and Spark
Yahoo compares Storm and Spark
 

Similar to Spark Streaming Recipes and "Exactly Once" Semantics Revised

Data Streaming in Kafka
Data Streaming in KafkaData Streaming in Kafka
Data Streaming in KafkaSilviuMarcu1
 
Learning spark ch10 - Spark Streaming
Learning spark ch10 - Spark StreamingLearning spark ch10 - Spark Streaming
Learning spark ch10 - Spark Streamingphanleson
 
Strata NYC 2015: What's new in Spark Streaming
Strata NYC 2015: What's new in Spark StreamingStrata NYC 2015: What's new in Spark Streaming
Strata NYC 2015: What's new in Spark StreamingDatabricks
 
Stefano Rocco, Roberto Bentivoglio - Scala in increasingly demanding environm...
Stefano Rocco, Roberto Bentivoglio - Scala in increasingly demanding environm...Stefano Rocco, Roberto Bentivoglio - Scala in increasingly demanding environm...
Stefano Rocco, Roberto Bentivoglio - Scala in increasingly demanding environm...Scala Italy
 
Scala in increasingly demanding environments - DATABIZ
Scala in increasingly demanding environments - DATABIZScala in increasingly demanding environments - DATABIZ
Scala in increasingly demanding environments - DATABIZDATABIZit
 
Seattle spark-meetup-032317
Seattle spark-meetup-032317Seattle spark-meetup-032317
Seattle spark-meetup-032317Nan Zhu
 
Spark streaming state of the union
Spark streaming state of the unionSpark streaming state of the union
Spark streaming state of the unionDatabricks
 
Java Performance and Profiling
Java Performance and ProfilingJava Performance and Profiling
Java Performance and ProfilingWSO2
 
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
 
Real time big data stream processing
Real time big data stream processing Real time big data stream processing
Real time big data stream processing Luay AL-Assadi
 
What no one tells you about writing a streaming app
What no one tells you about writing a streaming appWhat no one tells you about writing a streaming app
What no one tells you about writing a streaming apphadooparchbook
 
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...Spark Summit
 
Sherlock Homepage (Maarten Balliauw)
Sherlock Homepage (Maarten Balliauw)Sherlock Homepage (Maarten Balliauw)
Sherlock Homepage (Maarten Balliauw)Visug
 
Sherlock Homepage - A detective story about running large web services (VISUG...
Sherlock Homepage - A detective story about running large web services (VISUG...Sherlock Homepage - A detective story about running large web services (VISUG...
Sherlock Homepage - A detective story about running large web services (VISUG...Maarten Balliauw
 
Serverless London 2019 FaaS composition using Kafka and CloudEvents
Serverless London 2019   FaaS composition using Kafka and CloudEventsServerless London 2019   FaaS composition using Kafka and CloudEvents
Serverless London 2019 FaaS composition using Kafka and CloudEventsNeil Avery
 
Apache samza past, present and future
Apache samza  past, present and futureApache samza  past, present and future
Apache samza past, present and futureEd Yakabosky
 
Sherlock Homepage - A detective story about running large web services - NDC ...
Sherlock Homepage - A detective story about running large web services - NDC ...Sherlock Homepage - A detective story about running large web services - NDC ...
Sherlock Homepage - A detective story about running large web services - NDC ...Maarten Balliauw
 
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Rei...
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck -  Pravega: Storage Rei...Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck -  Pravega: Storage Rei...
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Rei...Flink Forward
 
Dataflow vs spark streaming
Dataflow vs spark streamingDataflow vs spark streaming
Dataflow vs spark streamingVenkatesh Iyer
 

Similar to Spark Streaming Recipes and "Exactly Once" Semantics Revised (20)

Data Streaming in Kafka
Data Streaming in KafkaData Streaming in Kafka
Data Streaming in Kafka
 
Learning spark ch10 - Spark Streaming
Learning spark ch10 - Spark StreamingLearning spark ch10 - Spark Streaming
Learning spark ch10 - Spark Streaming
 
Strata NYC 2015: What's new in Spark Streaming
Strata NYC 2015: What's new in Spark StreamingStrata NYC 2015: What's new in Spark Streaming
Strata NYC 2015: What's new in Spark Streaming
 
Stefano Rocco, Roberto Bentivoglio - Scala in increasingly demanding environm...
Stefano Rocco, Roberto Bentivoglio - Scala in increasingly demanding environm...Stefano Rocco, Roberto Bentivoglio - Scala in increasingly demanding environm...
Stefano Rocco, Roberto Bentivoglio - Scala in increasingly demanding environm...
 
Scala in increasingly demanding environments - DATABIZ
Scala in increasingly demanding environments - DATABIZScala in increasingly demanding environments - DATABIZ
Scala in increasingly demanding environments - DATABIZ
 
Seattle spark-meetup-032317
Seattle spark-meetup-032317Seattle spark-meetup-032317
Seattle spark-meetup-032317
 
Spark streaming state of the union
Spark streaming state of the unionSpark streaming state of the union
Spark streaming state of the union
 
Java Performance and Profiling
Java Performance and ProfilingJava Performance and Profiling
Java Performance and Profiling
 
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...
 
Real time big data stream processing
Real time big data stream processing Real time big data stream processing
Real time big data stream processing
 
What no one tells you about writing a streaming app
What no one tells you about writing a streaming appWhat no one tells you about writing a streaming app
What no one tells you about writing a streaming app
 
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
 
Sherlock Homepage (Maarten Balliauw)
Sherlock Homepage (Maarten Balliauw)Sherlock Homepage (Maarten Balliauw)
Sherlock Homepage (Maarten Balliauw)
 
Sherlock Homepage - A detective story about running large web services (VISUG...
Sherlock Homepage - A detective story about running large web services (VISUG...Sherlock Homepage - A detective story about running large web services (VISUG...
Sherlock Homepage - A detective story about running large web services (VISUG...
 
Serverless London 2019 FaaS composition using Kafka and CloudEvents
Serverless London 2019   FaaS composition using Kafka and CloudEventsServerless London 2019   FaaS composition using Kafka and CloudEvents
Serverless London 2019 FaaS composition using Kafka and CloudEvents
 
Apache samza past, present and future
Apache samza  past, present and futureApache samza  past, present and future
Apache samza past, present and future
 
Sherlock Homepage - A detective story about running large web services - NDC ...
Sherlock Homepage - A detective story about running large web services - NDC ...Sherlock Homepage - A detective story about running large web services - NDC ...
Sherlock Homepage - A detective story about running large web services - NDC ...
 
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Rei...
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck -  Pravega: Storage Rei...Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck -  Pravega: Storage Rei...
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Rei...
 
Dataflow vs spark streaming
Dataflow vs spark streamingDataflow vs spark streaming
Dataflow vs spark streaming
 
Spark streaming
Spark streamingSpark streaming
Spark streaming
 

Recently uploaded

Chinsurah Escorts ☎️8617697112 Starting From 5K to 15K High Profile Escorts ...
Chinsurah Escorts ☎️8617697112  Starting From 5K to 15K High Profile Escorts ...Chinsurah Escorts ☎️8617697112  Starting From 5K to 15K High Profile Escorts ...
Chinsurah Escorts ☎️8617697112 Starting From 5K to 15K High Profile Escorts ...Nitya salvi
 
Right Money Management App For Your Financial Goals
Right Money Management App For Your Financial GoalsRight Money Management App For Your Financial Goals
Right Money Management App For Your Financial GoalsJhone kinadey
 
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...SelfMade bd
 
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️Delhi Call girls
 
SHRMPro HRMS Software Solutions Presentation
SHRMPro HRMS Software Solutions PresentationSHRMPro HRMS Software Solutions Presentation
SHRMPro HRMS Software Solutions PresentationShrmpro
 
Exploring the Best Video Editing App.pdf
Exploring the Best Video Editing App.pdfExploring the Best Video Editing App.pdf
Exploring the Best Video Editing App.pdfproinshot.com
 
10 Trends Likely to Shape Enterprise Technology in 2024
10 Trends Likely to Shape Enterprise Technology in 202410 Trends Likely to Shape Enterprise Technology in 2024
10 Trends Likely to Shape Enterprise Technology in 2024Mind IT Systems
 
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfonteinmasabamasaba
 
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...masabamasaba
 
introduction-to-automotive Andoid os-csimmonds-ndctechtown-2021.pdf
introduction-to-automotive Andoid os-csimmonds-ndctechtown-2021.pdfintroduction-to-automotive Andoid os-csimmonds-ndctechtown-2021.pdf
introduction-to-automotive Andoid os-csimmonds-ndctechtown-2021.pdfVishalKumarJha10
 
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...harshavardhanraghave
 
%in ivory park+277-882-255-28 abortion pills for sale in ivory park
%in ivory park+277-882-255-28 abortion pills for sale in ivory park %in ivory park+277-882-255-28 abortion pills for sale in ivory park
%in ivory park+277-882-255-28 abortion pills for sale in ivory park masabamasaba
 
%in Harare+277-882-255-28 abortion pills for sale in Harare
%in Harare+277-882-255-28 abortion pills for sale in Harare%in Harare+277-882-255-28 abortion pills for sale in Harare
%in Harare+277-882-255-28 abortion pills for sale in Hararemasabamasaba
 
%in tembisa+277-882-255-28 abortion pills for sale in tembisa
%in tembisa+277-882-255-28 abortion pills for sale in tembisa%in tembisa+277-882-255-28 abortion pills for sale in tembisa
%in tembisa+277-882-255-28 abortion pills for sale in tembisamasabamasaba
 
VTU technical seminar 8Th Sem on Scikit-learn
VTU technical seminar 8Th Sem on Scikit-learnVTU technical seminar 8Th Sem on Scikit-learn
VTU technical seminar 8Th Sem on Scikit-learnAmarnathKambale
 
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️Delhi Call girls
 
Unlocking the Future of AI Agents with Large Language Models
Unlocking the Future of AI Agents with Large Language ModelsUnlocking the Future of AI Agents with Large Language Models
Unlocking the Future of AI Agents with Large Language Modelsaagamshah0812
 
The Top App Development Trends Shaping the Industry in 2024-25 .pdf
The Top App Development Trends Shaping the Industry in 2024-25 .pdfThe Top App Development Trends Shaping the Industry in 2024-25 .pdf
The Top App Development Trends Shaping the Industry in 2024-25 .pdfayushiqss
 

Recently uploaded (20)

Chinsurah Escorts ☎️8617697112 Starting From 5K to 15K High Profile Escorts ...
Chinsurah Escorts ☎️8617697112  Starting From 5K to 15K High Profile Escorts ...Chinsurah Escorts ☎️8617697112  Starting From 5K to 15K High Profile Escorts ...
Chinsurah Escorts ☎️8617697112 Starting From 5K to 15K High Profile Escorts ...
 
Right Money Management App For Your Financial Goals
Right Money Management App For Your Financial GoalsRight Money Management App For Your Financial Goals
Right Money Management App For Your Financial Goals
 
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
 
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
 
SHRMPro HRMS Software Solutions Presentation
SHRMPro HRMS Software Solutions PresentationSHRMPro HRMS Software Solutions Presentation
SHRMPro HRMS Software Solutions Presentation
 
Exploring the Best Video Editing App.pdf
Exploring the Best Video Editing App.pdfExploring the Best Video Editing App.pdf
Exploring the Best Video Editing App.pdf
 
10 Trends Likely to Shape Enterprise Technology in 2024
10 Trends Likely to Shape Enterprise Technology in 202410 Trends Likely to Shape Enterprise Technology in 2024
10 Trends Likely to Shape Enterprise Technology in 2024
 
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
 
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
 
introduction-to-automotive Andoid os-csimmonds-ndctechtown-2021.pdf
introduction-to-automotive Andoid os-csimmonds-ndctechtown-2021.pdfintroduction-to-automotive Andoid os-csimmonds-ndctechtown-2021.pdf
introduction-to-automotive Andoid os-csimmonds-ndctechtown-2021.pdf
 
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
Reassessing the Bedrock of Clinical Function Models: An Examination of Large ...
 
%in ivory park+277-882-255-28 abortion pills for sale in ivory park
%in ivory park+277-882-255-28 abortion pills for sale in ivory park %in ivory park+277-882-255-28 abortion pills for sale in ivory park
%in ivory park+277-882-255-28 abortion pills for sale in ivory park
 
CHEAP Call Girls in Pushp Vihar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Pushp Vihar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICECHEAP Call Girls in Pushp Vihar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Pushp Vihar (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
 
%in Harare+277-882-255-28 abortion pills for sale in Harare
%in Harare+277-882-255-28 abortion pills for sale in Harare%in Harare+277-882-255-28 abortion pills for sale in Harare
%in Harare+277-882-255-28 abortion pills for sale in Harare
 
%in tembisa+277-882-255-28 abortion pills for sale in tembisa
%in tembisa+277-882-255-28 abortion pills for sale in tembisa%in tembisa+277-882-255-28 abortion pills for sale in tembisa
%in tembisa+277-882-255-28 abortion pills for sale in tembisa
 
Microsoft AI Transformation Partner Playbook.pdf
Microsoft AI Transformation Partner Playbook.pdfMicrosoft AI Transformation Partner Playbook.pdf
Microsoft AI Transformation Partner Playbook.pdf
 
VTU technical seminar 8Th Sem on Scikit-learn
VTU technical seminar 8Th Sem on Scikit-learnVTU technical seminar 8Th Sem on Scikit-learn
VTU technical seminar 8Th Sem on Scikit-learn
 
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
 
Unlocking the Future of AI Agents with Large Language Models
Unlocking the Future of AI Agents with Large Language ModelsUnlocking the Future of AI Agents with Large Language Models
Unlocking the Future of AI Agents with Large Language Models
 
The Top App Development Trends Shaping the Industry in 2024-25 .pdf
The Top App Development Trends Shaping the Industry in 2024-25 .pdfThe Top App Development Trends Shaping the Industry in 2024-25 .pdf
The Top App Development Trends Shaping the Industry in 2024-25 .pdf
 

Spark Streaming Recipes and "Exactly Once" Semantics Revised

  • 1. Spark Streaming Spark streaming recipes and “exactly once” semantics revised
  • 3. Appsflyer as Marketing Platform Attribution Statistics: clicks, installs, in-app events, launches, uninstalls, etc. Life time value Retargeting Fraud detection Prediction A/B testing etc...
  • 4. Appsflyer Technology ~7B events / day Hundreds of machines in Amazon Tens of micro-services Apache Kafka service service service service service service DB Amazon S3 MongoDB Redshift Druid
  • 5. What is stream processing?
  • 6. Stream Processing Minimize latency between data ingestion and insights Usages ● Real-time dashboard ● Fraud prevention ● Ad bidding ● etc.
  • 7. Stream Processing Frameworks Key Differences ● Latency ● Windowing support ● Delivery semantics ● State management ● API easiness ● Programming languages support ● Community support ● etc..
  • 8. Apache Spark Spark Driver val textFile = sc.textFile("hdfs://...") val counts = textFile .flatMap(line => line.split(" ")) .map(word => (word, 1)) .reduceByKey(_ + _) counts.saveAsTextFile("hdfs://...") Cluster Manager Worker Node Executor Task Task Worker Node Executor Task Task
  • 9. Apache Spark External World RDD RDD RDD RDD RDD External World read read transform transform transform transform action
  • 10. Streaming in Spark Advantages ● Reuse existing infra ● Rich API ● Straightforward windowing ● It’s easier to implement “exactly once” Disadvantages ● Latency Input Stream Micro batches Spark Engine Processed data
  • 11. Windowing in Spark Streaming ● Window length and sliding interval must be multiples of batch interval ● Possible usages ○ Finding Top N elements during last M period of time ○ Pre-aggregation of data prior to inserting to DB ○ etc. DStream Window length Sliding interval
  • 12. Do we need “exactly once” semantics?
  • 13. Data Processing Paradigms Batch layer Real-time layer Real-time layer http://www.kappa-architecture.com/https://en.wikipedia.org/wiki/Lambda_architecture
  • 14. How do we achieve “exactly once”?
  • 15. Achieving “Exactly once” Producer Doesn’t duplicate messages Stream processor Tracking state (checkpointing) Resilient components Consumer Reads only new messages “Easy” way: Message deduplication based on some ID Idempotent output destination
  • 16. Stream Checkpointing https://en.wikipedia.org/wiki/Snapshot_algorithm Barriers are injected into the data stream. Once intermediate step sees barriers from all of its input streams it outputs barrier to all of its outgoing streams. Once all sink operators see barrier for a snapshot, they acknowledge the snapshot, and it’s considered committed. Multiple barriers can be seen in stream flow. Operators store their state to an external storage. storage
  • 17. Micro-batch Checkpointing receive process state receive process state receive process state while (true) { // 1. receive next batch of data // 2. compute next stream and state } Unit of fault tolerance
  • 18. Resilience in Spark Streaming All Spark components must be resilient! Driver application process Master process Worker process Executor process Receiver thread Worker node Driver Master Worker Node Executor Tas k Tas k Worker Node Executor Tas k Tas k
  • 19. Driver Driver Resilience Client mode Driver application is running inside the “spark-submit” process. If this process dies the entire application is killed. Cluster mode Driver application runs on one of worker nodes. “--supervise” option makes driver restart on a different worker node. Running through Marathon Marathon can re-start failed applications automatically. Master Worker Node Executor Tas k Tas k Worker Node Executor Tas k Tas k
  • 20. Master Resilience Single master The entire application is killed. Multi-master mode A standby master is elected active. Worker nodes automatically register with new master. Leader election via ZooKeeper. Driver Master Worker Node Executor Tas k Tas k Worker Node Executor Tas k Tas k
  • 21. Worker Resilience Worker process When failed, all child processes (driver or executor) are killed. New worker process is launched automatically. Executor process Restarted on failure by the parent worker process. Receiver thread Running inside the Executor process - same as Executor. Worker node Driver Master Worker Node Executor Tas k Tas k Worker Node Executor Tas k Tas k
  • 22. Resilience doesn’t ensure “exactly once”
  • 23. Checkpointing Checkpointing helps recover from driver failure. Stores computation graph to some fault tolerant place (like HDFS or S3). What is saved as metadata Metadata of queued but not processed batches Stream operations (code) Configuration Disadvantages Frequent checkpointing reduces throughput.
  • 24. Write Ahead Log Synchronously saves received data to fault tolerant storage. Helps recover received, but not yet committed blocks. Disadvantages Additional storage is required. Reduced throughput. Executor input stream Receiver
  • 25. Problems with Checkpointing and WAL Data can be lost even when using checkpointing (batches hold in memory will be lost on driver failure). Checkpointing and WAL prevent data loss, but do not provide “exactly once” semantics. If receiver fails before updating offsets in ZooKeeper - we are in trouble. In this case data will be re-read from Kafka and from WAL. Still not exactly once!
  • 26. The Solution Don’t use receivers - read directly from input stream instead. Driver instructs executors what range to read from a stream (stream must be rewindable). Read range is attached to the batch itself. Example (Kafka direct stream): Application Driver Streaming Context 1. Periodically query latest offsets for topics & partitions 2. Calculates offset ranges for the next batch Executor 3. Schedule the next micro- batch job 4. Consume data for the calculated offsets
  • 28. The Problem Events counting. Group by different set of dimensions. Have pre-aggregation layer that reduces load on DB on spikes. DB app_id event_name country count com.app.bla FIRST_LAUNCH US 152 com.app.bla purchase IL 10 com.app.jo custom_inapp_20 US 45
  • 29. Transactional Events Aggregator Based on SQL database Store Kafka partition offsets into the DB Increment event counters in transaction based on current and stored offsets. SQL DB Driver Executor Executor 1. Read last Kafka partitions and their offsets from the DB 2. Create direct Kafka stream based on read partitions and offsets 3. Consume events from Kafka 4. Aggregate events 5. Upsert event counter along with current offsets in transaction
  • 33. Snapshotting Events Aggregator Driver Executor Executor 1. Read last Kafka partitions and their offsets from S3 2. Create direct Kafka stream based on read partitions and offsets 3. Consume events from Kafka 4. Aggregate events 5. Store processed data and Kafka offsets under /data/ts=<timestamp> and /offsets/ts=<timestamp> respectively S3 Aggregator Application
  • 34. Snapshotting Events Aggregator Executor Executor Executor 1. Find last committed timestamp 2. Read data for the last timestamp from /data/ts=<timestamp> 4. Aggregate events by different dimensions, and split to cubes 6. Delete offsets and data for the timestamp /offsets/ts=<timestamp> /data/ts=<timestamp> S3 Loader Application Cassandra 5. Increment counters in different cubes Driver
  • 38. Deployment We use Mesos Master HA for free. Marathon keeps Spark streaming application alive. Read carefully http://spark.apache.org/docs/latest/streaming-programming-guide.html#performance-tuning Inspect, re-configure, retry Turn off Spark dynamicity Preserve data locality Find balance between cores/batch interval/block interval Processing time must be less than batch interval Tips
  • 40. Right Now Real time analytics dashboard
  • 41. Right Now Processes ~50M events a day Reduces the stream in two sliding windows: 1. Last 5 seconds (“now”) 2. Last 10 minutes (“recent”) At most once semantics
  • 43. Right Now Why Spark? Experienced with Spark Convenient Clojure wrappers (Sparkling, Flambo) Documentation and community
  • 44. Right Now In Production 3 m3.xlarge machines for the workers (4 cores each) spark.default.parallelism=10 Lesson learned: foreachRDD and foreachPartition