SlideShare une entreprise Scribd logo
1  sur  59
Télécharger pour lire hors ligne
Four Things to Know about
Reliable Spark Streaming
Dean Wampler, Typesafe
Tathagata Das, Databricks
Agenda for today
• The Stream Processing Landscape
• How Spark Streaming Works - A Quick Overview
• Features in Spark Streaming that Help Prevent
Data Loss
• Design Tips for Successful Streaming
Applications
The Stream
Processing Landscape
Stream Processors
Stream Storage
Stream Sources
MQTT$
How Spark Streaming Works:
A Quick Overview
Spark Streaming
Scalable, fault-tolerant stream processing system
File systems
Databases
Dashboards
Flume
Kinesis
HDFS/S3
Kafka
Twitter
High-level API
joins, windows, …
often 5x less code
Fault-tolerant
Exactly-once semantics,
even for stateful ops
Integration
Integrates with MLlib, SQL,
DataFrames, GraphX
Spark Streaming
Receivers receive data streams and chop them up into batches
Spark processes the batches and pushes out the results
9
data streams
receivers
batches results
Word Count with Kafka
val!context!=!new!StreamingContext(conf,!Seconds(1))!
val!lines!=!KafkaUtils.createStream(context,!...)!
10
entry point of streaming
functionality
create DStream
from Kafka data
Word Count with Kafka
val!context!=!new!StreamingContext(conf,!Seconds(1))!
val!lines!=!KafkaUtils.createStream(context,!...)!
val!words!=!lines.flatMap(_.split("!"))!
11
split lines into words
Word Count with Kafka
val!context!=!new!StreamingContext(conf,!Seconds(1))!
val!lines!=!KafkaUtils.createStream(context,!...)!
val!words!=!lines.flatMap(_.split("!"))!
val!wordCounts!=!words.map(x!=>!(x,!1))!
!!!!!!!!!!!!!!!!!!!!!!.reduceByKey(_!+!_)!
wordCounts.print()!
context.start()!
12
print some counts on screen
count the words
start receiving and
transforming the data
Word Count with Kafka
object!WordCount!{!
!!def!main(args:!Array[String])!{!
!!!!val!context!=!new!StreamingContext(new!SparkConf(),!Seconds(1))!
!!!!val!lines!=!KafkaUtils.createStream(context,!...)!
!!!!val!words!=!lines.flatMap(_.split("!"))!
!!!!val!wordCounts!=!words.map(x!=>!(x,1)).reduceByKey(_!+!_)!
!!!!wordCounts.print()!
!!!!context.start()!
!!!!context.awaitTermination()!
!!}!
}!
13
Features in Spark Streaming
that Help Prevent Data Loss
15
A Deeper View of
Spark Streaming
Any Spark Application
16
Spark
Driver
User code runs in
the driver process
YARN / Mesos /
Spark Standalone
cluster
Any Spark Application
17
Spark
Driver
User code runs in
the driver process
YARN / Mesos /
Spark Standalone
cluster
Spark
Executor
Spark
Executor
Spark
Executor
Driver launches
executors in
cluster
Any Spark Application
18
Spark
Driver
User code runs in
the driver process
YARN / Mesos /
Spark Standalone
cluster
Tasks sent to
executors for
processing data
Spark
Executor
Spark
Executor
Spark
Executor
Driver launches
executors in
cluster
Spark Streaming Application: Receive data
19
Executor
Executor
Driver runs
receivers as long
running tasks Receiver Data stream
Driver@
object!WordCount!{!
!!def!main(args:!Array[String])!{!
!!!!val!context!=!new!StreamingContext(...)!
!!!!val!lines!=!KafkaUtils.createStream(...)!
!!!!val!words!=!lines.flatMap(_.split("!"))!
!!!!val!wordCounts!=!words.map(x!=>!(x,1))!
!!!!!!!!!!!!!!!!!!!!!!!!!!.reduceByKey(_!+!_)!
!!!!wordCounts.print()!
!!!!context.start()!
!!!!context.awaitTermination()!
!!}!
}!
Spark Streaming Application: Receive data
20
Executor
Executor
Driver runs
receivers as long
running tasks Receiver Data stream
Driver@
object!WordCount!{!
!!def!main(args:!Array[String])!{!
!!!!val!context!=!new!StreamingContext(...)!
!!!!val!lines!=!KafkaUtils.createStream(...)!
!!!!val!words!=!lines.flatMap(_.split("!"))!
!!!!val!wordCounts!=!words.map(x!=>!(x,1))!
!!!!!!!!!!!!!!!!!!!!!!!!!!.reduceByKey(_!+!_)!
!!!!wordCounts.print()!
!!!!context.start()!
!!!!context.awaitTermination()!
!!}!
}!
Receiver divides
stream into blocks and
keeps in memory
Data Blocks!
Spark Streaming Application: Receive data
21
Executor
Executor
Driver runs
receivers as long
running tasks Receiver Data stream
Driver@
object!WordCount!{!
!!def!main(args:!Array[String])!{!
!!!!val!context!=!new!StreamingContext(...)!
!!!!val!lines!=!KafkaUtils.createStream(...)!
!!!!val!words!=!lines.flatMap(_.split("!"))!
!!!!val!wordCounts!=!words.map(x!=>!(x,1))!
!!!!!!!!!!!!!!!!!!!!!!!!!!.reduceByKey(_!+!_)!
!!!!wordCounts.print()!
!!!!context.start()!
!!!!context.awaitTermination()!
!!}!
}!
Receiver divides
stream into blocks and
keeps in memory
Data Blocks!
Blocks also
replicated to
another executor Data Blocks!
Spark Streaming Application: Process data
22
Executor
Executor
Receiver
Data Blocks!
Data Blocks!
Every batch
interval, driver
launches tasks to
process the blocks
Driver@
object!WordCount!{!
!!def!main(args:!Array[String])!{!
!!!!val!context!=!new!StreamingContext(...)!
!!!!val!lines!=!KafkaUtils.createStream(...)!
!!!!val!words!=!lines.flatMap(_.split("!"))!
!!!!val!wordCounts!=!words.map(x!=>!(x,1))!
!!!!!!!!!!!!!!!!!!!!!!!!!!.reduceByKey(_!+!_)!
!!!!wordCounts.print()!
!!!!context.start()!
!!!!context.awaitTermination()!
!!}!
}!
Spark Streaming Application: Process data
23
Executor
Executor
Receiver
Data Blocks!
Data Blocks!
Data
store
Every batch
interval, driver
launches tasks to
process the blocks
Driver@
object!WordCount!{!
!!def!main(args:!Array[String])!{!
!!!!val!context!=!new!StreamingContext(...)!
!!!!val!lines!=!KafkaUtils.createStream(...)!
!!!!val!words!=!lines.flatMap(_.split("!"))!
!!!!val!wordCounts!=!words.map(x!=>!(x,1))!
!!!!!!!!!!!!!!!!!!!!!!!!!!.reduceByKey(_!+!_)!
!!!!wordCounts.print()!
!!!!context.start()!
!!!!context.awaitTermination()!
!!}!
}!
Fault Tolerance
and Reliability
Failures? Why care?
Many streaming applications need zero data loss guarantees
despite any kind of failures in the system
At least once guarantee – every record processed at least once
Exactly once guarantee – every record processed exactly once
Different kinds of failures – executor and driver
Some failures and guarantee requirements need additional
configurations and setups
25
Executor
Receiver
Data Blocks!
What if an executor fails?
26
Executor
Failed Ex.
Receiver
Blocks!
Blocks!
Driver
If executor fails,
receiver is lost and
all blocks are lost
Executor
Receiver
Data Blocks!
What if an executor fails?
Tasks and receivers restarted by Spark
automatically, no config needed
27
Executor
Failed Ex.
Receiver
Blocks!
Blocks!
Driver
If executor fails,
receiver is lost and
all blocks are lost
Receiver
Receiver
restarted
Tasks restarted
on block replicas
What if the driver fails?
28
Executor
Blocks!How do we
recover?
When the driver
fails, all the
executors fail
All computation,
all received
blocks are lost
Executor
Receiver
Blocks!
Failed Ex.
Receiver
Blocks!
Failed
Executor
Blocks!
Driver
Failed
Driver
Recovering Driver w/ DStream Checkpointing
DStream Checkpointing:
Periodically save the DAG of DStreams to
fault-tolerant storage
29
Executor
Blocks!
Executor
Receiver
Blocks!
Active
Driver
Checkpoint info
to HDFS / S3
Recovering Driver w/ DStream Checkpointing
30
Failed driver can be restarted
from checkpoint information
Failed
Driver
Restarted
Driver
DStream Checkpointing:
Periodically save the DAG of DStreams to
fault-tolerant storage
Recovering Driver w/ DStream Checkpointing
31
Failed driver can be restarted
from checkpoint information
Failed
Driver
Restarted
Driver
New
Executor
New
Executor
Receiver
New executors
launched and
receivers
restarted
DStream Checkpointing:
Periodically save the DAG of DStreams to
fault-tolerant storage
Recovering Driver w/ DStream Checkpointing
1.  Configure automatic driver restart
All cluster managers support this
2.  Set a checkpoint directory in a HDFS-compatible file
system
!streamingContext.checkpoint(hdfsDirectory)!
3.  Slightly restructure of the code to use checkpoints for
recovery
32
Configurating Automatic Driver Restart
Spark Standalone – Use spark-submit with “cluster” mode and “--
supervise”
See http://spark.apache.org/docs/latest/spark-standalone.html
YARN – Use spark-submit in “cluster” mode
See YARN config “yarn.resourcemanager.am.max-attempts”
Mesos – Marathon can restart applications or use the “--supervise” flag.
33
Restructuring code for Checkpointing
34
val!context!=!new!StreamingContext(...)!
val!lines!=!KafkaUtils.createStream(...)!
val!words!=!lines.flatMap(...)!
...!
context.start()!
Create
+
Setup
Start
Restructuring code for Checkpointing
35
val!context!=!new!StreamingContext(...)!
val!lines!=!KafkaUtils.createStream(...)!
val!words!=!lines.flatMap(...)!
...!
context.start()!
Create
+
Setup
Start
def!creatingFunc():!StreamingContext!=!{!!
!!!val!context!=!new!StreamingContext(...)!!!
!!!val!lines!=!KafkaUtils.createStream(...)!
!!!val!words!=!lines.flatMap(...)!
!!!...!
!!!context.checkpoint(hdfsDir)!
}!
Put all setup code into a function that returns a new StreamingContext
Restructuring code for Checkpointing
36
val!context!=!new!StreamingContext(...)!
val!lines!=!KafkaUtils.createStream(...)!
val!words!=!lines.flatMap(...)!
...!
context.start()!
Create
+
Setup
Start
def!creatingFunc():!StreamingContext!=!{!!
!!!val!context!=!new!StreamingContext(...)!!!
!!!val!lines!=!KafkaUtils.createStream(...)!
!!!val!words!=!lines.flatMap(...)!
!!!...!
!!!context.checkpoint(hdfsDir)!
}!
Put all setup code into a function that returns a new StreamingContext
Get context setup from HDFS dir OR create a new one with the function
val!context!=!
StreamingContext.getOrCreate(!
!!hdfsDir,!creatingFunc)!
context.start()!
Restructuring code for Checkpointing
StreamingContext.getOrCreate():
If HDFS directory has checkpoint info
recover context from info
else
call creatingFunc() to create
and setup a new context
Restarted process can figure out whether
to recover using checkpoint info or not
37
def!creatingFunc():!StreamingContext!=!{!!
!!!val!context!=!new!StreamingContext(...)!!!
!!!val!lines!=!KafkaUtils.createStream(...)!
!!!val!words!=!lines.flatMap(...)!
!!!...!
!!!context.checkpoint(hdfsDir)!
}!
val!context!=!
StreamingContext.getOrCreate(!
!!hdfsDir,!creatingFunc)!
context.start()!
Received blocks lost on Restart!
38
Failed
Driver
Restarted
Driver
New
Executor
New Ex.
Receiver
No Blocks!
In-memory blocks of
buffered data are
lost on driver restart
Recovering data with Write Ahead Logs
Write Ahead Log (WAL): Synchronously
save received data to fault-tolerant storage
39
Executor
Blocks saved
to HDFS
Executor
Receiver
Blocks!
Active
Driver
Data stream
Recovering data with Write Ahead Logs
40
Failed
Driver
Restarted
Driver
New
Executor
New Ex.
Receiver
Blocks!
Blocks recovered
from Write Ahead Log
Write Ahead Log (WAL): Synchronously
save received data to fault-tolerant storage
Recovering data with Write Ahead Logs
1.  Enable checkpointing, logs written in checkpoint directory
3.  Enabled WAL in SparkConf configuration
!!!!!sparkConf.set("spark.streaming.receiver.writeAheadLog.enable",!"true")!
3.  Receiver should also be reliable
Acknowledge source only after data saved to WAL
Unacked data will be replayed from source by restarted receiver
5.  Disable in-memory replication (already replicated by HDFS)
!Use StorageLevel.MEMORY_AND_DISK_SER for input DStreams
41
RDD Checkpointing
•  Stateful stream processing can lead to long RDD lineages
•  Long lineage = bad for fault-tolerance, too much recomputation
•  RDD checkpointing saves RDD data to the fault-tolerant storage to
limit lineage and recomputation
•  More:
http://spark.apache.org/docs/latest/streaming-programming-guide.html#checkpointing
42
Fault-tolerance Semantics
43
Zero data loss = every stage processes each
event at least once despite any failure
Sources
Transforming
Sinks
Outputting
Receiving
Fault-tolerance Semantics
44
Sources
Transforming
Sinks
Outputting
Receiving
Exactly once, as long as received data is not lost
Receiving
Outputting
End-to-end semantics:
At-least once
Fault-tolerance Semantics
45
Sources
Transforming
Sinks
Outputting
Receiving
Exactly once, as long as received data is not lost
Receiving
Outputting
Exactly once, if outputs are idempotent or transactional
End-to-end semantics:
At-least once
Fault-tolerance Semantics
46
Sources
Transforming
Sinks
Outputting
Receiving
Exactly once, as long as received data is not lost
At least once, w/ Checkpointing + WAL + Reliable receiversReceiving
Outputting
Exactly once, if outputs are idempotent or transactional
End-to-end semantics:
At-least once
Fault-tolerance Semantics
47
Exactly once receiving with new Kafka Direct approach
Treats Kafka like a replicated log, reads it like a file
Does not use receivers
No need to create multiple DStreams and union them
No need to enable Write Ahead Logs
!
@val!directKafkaStream!=!KafkaUtils.createDirectStream(...)!
!
https://databricks.com/blog/2015/03/30/improvements-to-kafka-integration-of-spark-streaming.html
http://spark.apache.org/docs/latest/streaming-kafka-integration.html
Sources
Transforming
Sinks
Outputting
Receiving
Fault-tolerance Semantics
48
Exactly once receiving with new Kafka Direct approach
Sources
Transforming
Sinks
Outputting
Receiving
Exactly once, as long as received data is not lost
Exactly once, if outputs are idempotent or transactional
End-to-end semantics:
Exactly once!
Design Tips for Successful
Streaming Applications
Areas for consideration
• Enhance resilience with additional
components.
• Mini-batch vs. per-message handling.
• Exploit Reactive Streams.
• Use Storm, Akka, Samza, etc. for handling
individual messages, especially with sub-
second latency requirements.
• Use Spark Streaming’s mini-batch model for
the Lambda architecture and highly-scalable
analytics.
Mini-batch vs. per-message handling
• Consider Kafka or Kinesis for resilient buffering
in front of Spark Streaming.
•  Buffer for traffic spikes.
•  Re-retrieval of data if an RDD partition is lost and must be
reconstructed from the source.
• Going to store the raw data anyway?
•  Do it first, then ingest to Spark from that storage.
Enhance Resiliency with Additional
Components.
• Spark Streaming v1.5 will have support for back pressure to more
easily build end-to-end reactive applications
Exploit Reactive Streams
• Spark Streaming v1.5 will have support for back
pressure to more easily build end-to-end reactive
applications
• Backpressure from consumer to producer:
•  Prevents buffer overflows.
•  Avoids unnecessary throttling.
Exploit Reactive Streams
• Spark Streaming v1.4? Buffer with Akka
Streams:
Exploit Reactive Streams
Event
Source
Akka
App
Spark
Streaming App
Event
Event
Event
Event
feedback
(back pressure)
• Spark Streaming v1.4 has a rate limit property:
•  spark.streaming.receiver.maxRate
•  Consider setting it for long-running streaming apps with a
variable input flow rate.
• Have a graph of Reactive Streams? Consider using an
Akka app to buffer the data fed to Spark Streaming over
a socket (until 1.5…).
Exploit Reactive Streams
Thank you!
Dean Wampler, Typesafe
Tathagata Das, Databricks
Databricks is
available as a hosted
platform on 
AWS with a monthly
subscription. 
What to do next?
Start with a free trial
Typesafe now offers
certified support for
Spark, Mesos &
DCOS, read more
about it
READ MORE
REACTIVE PLATFORM

Full Lifecycle Support for Play, Akka, Scala and Spark
Give your project a boost with Reactive Platform:
• Monitor Message-Driven Apps
• Resolve Network Partitions Decisively
• Integrate Easily with Legacy Systems
• Eliminate Incompatibility & Security Risks
• Protect Apps Against Abuse
• Expert Support from Dedicated Product Teams
Enjoy learning? See about the availability of
on-site training for Scala, Akka, Play and Spark!
Learn more about our offersCONTACT US

Contenu connexe

Tendances

Evolution from EDA to Data Mesh: Data in Motion
Evolution from EDA to Data Mesh: Data in MotionEvolution from EDA to Data Mesh: Data in Motion
Evolution from EDA to Data Mesh: Data in Motionconfluent
 
Serverless Kafka and Spark in a Multi-Cloud Lakehouse Architecture
Serverless Kafka and Spark in a Multi-Cloud Lakehouse ArchitectureServerless Kafka and Spark in a Multi-Cloud Lakehouse Architecture
Serverless Kafka and Spark in a Multi-Cloud Lakehouse ArchitectureKai Wähner
 
Introduction SQL Analytics on Lakehouse Architecture
Introduction SQL Analytics on Lakehouse ArchitectureIntroduction SQL Analytics on Lakehouse Architecture
Introduction SQL Analytics on Lakehouse ArchitectureDatabricks
 
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Databricks
 
Introduction to apache spark
Introduction to apache spark Introduction to apache spark
Introduction to apache spark Aakashdata
 
Intro to databricks delta lake
 Intro to databricks delta lake Intro to databricks delta lake
Intro to databricks delta lakeMykola Zerniuk
 
Designing and Building Next Generation Data Pipelines at Scale with Structure...
Designing and Building Next Generation Data Pipelines at Scale with Structure...Designing and Building Next Generation Data Pipelines at Scale with Structure...
Designing and Building Next Generation Data Pipelines at Scale with Structure...Databricks
 
Data Quality With or Without Apache Spark and Its Ecosystem
Data Quality With or Without Apache Spark and Its EcosystemData Quality With or Without Apache Spark and Its Ecosystem
Data Quality With or Without Apache Spark and Its EcosystemDatabricks
 
[DSC Europe 22] Overview of the Databricks Platform - Petar Zecevic
[DSC Europe 22] Overview of the Databricks Platform - Petar Zecevic[DSC Europe 22] Overview of the Databricks Platform - Petar Zecevic
[DSC Europe 22] Overview of the Databricks Platform - Petar ZecevicDataScienceConferenc1
 
How to build a streaming Lakehouse with Flink, Kafka, and Hudi
How to build a streaming Lakehouse with Flink, Kafka, and HudiHow to build a streaming Lakehouse with Flink, Kafka, and Hudi
How to build a streaming Lakehouse with Flink, Kafka, and HudiFlink Forward
 
Apache Spark sql
Apache Spark sqlApache Spark sql
Apache Spark sqlaftab alam
 
Apache Spark Tutorial | Spark Tutorial for Beginners | Apache Spark Training ...
Apache Spark Tutorial | Spark Tutorial for Beginners | Apache Spark Training ...Apache Spark Tutorial | Spark Tutorial for Beginners | Apache Spark Training ...
Apache Spark Tutorial | Spark Tutorial for Beginners | Apache Spark Training ...Edureka!
 
Modern Data architecture Design
Modern Data architecture DesignModern Data architecture Design
Modern Data architecture DesignKujambu Murugesan
 
Lambda Architecture in the Cloud with Azure Databricks with Andrei Varanovich
Lambda Architecture in the Cloud with Azure Databricks with Andrei VaranovichLambda Architecture in the Cloud with Azure Databricks with Andrei Varanovich
Lambda Architecture in the Cloud with Azure Databricks with Andrei VaranovichDatabricks
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Databricks
 
Simplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache SparkSimplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache SparkDatabricks
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiDatabricks
 
Programming in Spark using PySpark
Programming in Spark using PySpark      Programming in Spark using PySpark
Programming in Spark using PySpark Mostafa
 
Data Distribution and Ordering for Efficient Data Source V2
Data Distribution and Ordering for Efficient Data Source V2Data Distribution and Ordering for Efficient Data Source V2
Data Distribution and Ordering for Efficient Data Source V2Databricks
 

Tendances (20)

Evolution from EDA to Data Mesh: Data in Motion
Evolution from EDA to Data Mesh: Data in MotionEvolution from EDA to Data Mesh: Data in Motion
Evolution from EDA to Data Mesh: Data in Motion
 
Serverless Kafka and Spark in a Multi-Cloud Lakehouse Architecture
Serverless Kafka and Spark in a Multi-Cloud Lakehouse ArchitectureServerless Kafka and Spark in a Multi-Cloud Lakehouse Architecture
Serverless Kafka and Spark in a Multi-Cloud Lakehouse Architecture
 
Introduction SQL Analytics on Lakehouse Architecture
Introduction SQL Analytics on Lakehouse ArchitectureIntroduction SQL Analytics on Lakehouse Architecture
Introduction SQL Analytics on Lakehouse Architecture
 
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
 
Introduction to apache spark
Introduction to apache spark Introduction to apache spark
Introduction to apache spark
 
Intro to databricks delta lake
 Intro to databricks delta lake Intro to databricks delta lake
Intro to databricks delta lake
 
Designing and Building Next Generation Data Pipelines at Scale with Structure...
Designing and Building Next Generation Data Pipelines at Scale with Structure...Designing and Building Next Generation Data Pipelines at Scale with Structure...
Designing and Building Next Generation Data Pipelines at Scale with Structure...
 
Data Quality With or Without Apache Spark and Its Ecosystem
Data Quality With or Without Apache Spark and Its EcosystemData Quality With or Without Apache Spark and Its Ecosystem
Data Quality With or Without Apache Spark and Its Ecosystem
 
[DSC Europe 22] Overview of the Databricks Platform - Petar Zecevic
[DSC Europe 22] Overview of the Databricks Platform - Petar Zecevic[DSC Europe 22] Overview of the Databricks Platform - Petar Zecevic
[DSC Europe 22] Overview of the Databricks Platform - Petar Zecevic
 
How to build a streaming Lakehouse with Flink, Kafka, and Hudi
How to build a streaming Lakehouse with Flink, Kafka, and HudiHow to build a streaming Lakehouse with Flink, Kafka, and Hudi
How to build a streaming Lakehouse with Flink, Kafka, and Hudi
 
Apache Spark sql
Apache Spark sqlApache Spark sql
Apache Spark sql
 
Apache Spark Tutorial | Spark Tutorial for Beginners | Apache Spark Training ...
Apache Spark Tutorial | Spark Tutorial for Beginners | Apache Spark Training ...Apache Spark Tutorial | Spark Tutorial for Beginners | Apache Spark Training ...
Apache Spark Tutorial | Spark Tutorial for Beginners | Apache Spark Training ...
 
Modern Data architecture Design
Modern Data architecture DesignModern Data architecture Design
Modern Data architecture Design
 
Lambda Architecture in the Cloud with Azure Databricks with Andrei Varanovich
Lambda Architecture in the Cloud with Azure Databricks with Andrei VaranovichLambda Architecture in the Cloud with Azure Databricks with Andrei Varanovich
Lambda Architecture in the Cloud with Azure Databricks with Andrei Varanovich
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
 
Simplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache SparkSimplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache Spark
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and Hudi
 
Programming in Spark using PySpark
Programming in Spark using PySpark      Programming in Spark using PySpark
Programming in Spark using PySpark
 
Apache Spark Overview
Apache Spark OverviewApache Spark Overview
Apache Spark Overview
 
Data Distribution and Ordering for Efficient Data Source V2
Data Distribution and Ordering for Efficient Data Source V2Data Distribution and Ordering for Efficient Data Source V2
Data Distribution and Ordering for Efficient Data Source V2
 

En vedette

Alpine academy apache spark series #1 introduction to cluster computing wit...
Alpine academy apache spark series #1   introduction to cluster computing wit...Alpine academy apache spark series #1   introduction to cluster computing wit...
Alpine academy apache spark series #1 introduction to cluster computing wit...Holden Karau
 
Rethinking Streaming Analytics For Scale
Rethinking Streaming Analytics For ScaleRethinking Streaming Analytics For Scale
Rethinking Streaming Analytics For ScaleHelena Edelson
 
Lambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, Scala
Lambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, ScalaLambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, Scala
Lambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, ScalaHelena Edelson
 
Reactive dashboard’s using apache spark
Reactive dashboard’s using apache sparkReactive dashboard’s using apache spark
Reactive dashboard’s using apache sparkRahul Kumar
 
Reactive app using actor model & apache spark
Reactive app using actor model & apache sparkReactive app using actor model & apache spark
Reactive app using actor model & apache sparkRahul Kumar
 
Real-Time Anomaly Detection with Spark MLlib, Akka and Cassandra
Real-Time Anomaly Detection  with Spark MLlib, Akka and  CassandraReal-Time Anomaly Detection  with Spark MLlib, Akka and  Cassandra
Real-Time Anomaly Detection with Spark MLlib, Akka and CassandraNatalino Busa
 
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 AkkaHelena Edelson
 
Intro to Apache Spark
Intro to Apache SparkIntro to Apache Spark
Intro to Apache SparkMammoth Data
 
Spark Kernel Talk - Apache Spark Meetup San Francisco (July 2015)
Spark Kernel Talk - Apache Spark Meetup San Francisco (July 2015)Spark Kernel Talk - Apache Spark Meetup San Francisco (July 2015)
Spark Kernel Talk - Apache Spark Meetup San Francisco (July 2015)Robert "Chip" Senkbeil
 
Sa introduction to big data pipelining with cassandra & spark west mins...
Sa introduction to big data pipelining with cassandra & spark   west mins...Sa introduction to big data pipelining with cassandra & spark   west mins...
Sa introduction to big data pipelining with cassandra & spark west mins...Simon Ambridge
 
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)Helena Edelson
 
Using Spark, Kafka, Cassandra and Akka on Mesos for Real-Time Personalization
Using Spark, Kafka, Cassandra and Akka on Mesos for Real-Time PersonalizationUsing Spark, Kafka, Cassandra and Akka on Mesos for Real-Time Personalization
Using Spark, Kafka, Cassandra and Akka on Mesos for Real-Time PersonalizationPatrick Di Loreto
 
Data Science lifecycle with Apache Zeppelin and Spark by Moonsoo Lee
Data Science lifecycle with Apache Zeppelin and Spark by Moonsoo LeeData Science lifecycle with Apache Zeppelin and Spark by Moonsoo Lee
Data Science lifecycle with Apache Zeppelin and Spark by Moonsoo LeeSpark Summit
 
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...Helena Edelson
 
Data processing platforms architectures with Spark, Mesos, Akka, Cassandra an...
Data processing platforms architectures with Spark, Mesos, Akka, Cassandra an...Data processing platforms architectures with Spark, Mesos, Akka, Cassandra an...
Data processing platforms architectures with Spark, Mesos, Akka, Cassandra an...Anton Kirillov
 
Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015Evan Chan
 
Deep Recurrent Neural Networks for Sequence Learning in Spark by Yves Mabiala
Deep Recurrent Neural Networks for Sequence Learning in Spark by Yves MabialaDeep Recurrent Neural Networks for Sequence Learning in Spark by Yves Mabiala
Deep Recurrent Neural Networks for Sequence Learning in Spark by Yves MabialaSpark Summit
 
Consumer offset management in Kafka
Consumer offset management in KafkaConsumer offset management in Kafka
Consumer offset management in KafkaJoel Koshy
 

En vedette (20)

Alpine academy apache spark series #1 introduction to cluster computing wit...
Alpine academy apache spark series #1   introduction to cluster computing wit...Alpine academy apache spark series #1   introduction to cluster computing wit...
Alpine academy apache spark series #1 introduction to cluster computing wit...
 
Rethinking Streaming Analytics For Scale
Rethinking Streaming Analytics For ScaleRethinking Streaming Analytics For Scale
Rethinking Streaming Analytics For Scale
 
Lambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, Scala
Lambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, ScalaLambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, Scala
Lambda Architecture with Spark Streaming, Kafka, Cassandra, Akka, Scala
 
Reactive dashboard’s using apache spark
Reactive dashboard’s using apache sparkReactive dashboard’s using apache spark
Reactive dashboard’s using apache spark
 
Reactive app using actor model & apache spark
Reactive app using actor model & apache sparkReactive app using actor model & apache spark
Reactive app using actor model & apache spark
 
How to deploy Apache Spark 
to Mesos/DCOS
How to deploy Apache Spark 
to Mesos/DCOSHow to deploy Apache Spark 
to Mesos/DCOS
How to deploy Apache Spark 
to Mesos/DCOS
 
Real-Time Anomaly Detection with Spark MLlib, Akka and Cassandra
Real-Time Anomaly Detection  with Spark MLlib, Akka and  CassandraReal-Time Anomaly Detection  with Spark MLlib, Akka and  Cassandra
Real-Time Anomaly Detection with Spark MLlib, Akka and Cassandra
 
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
 
Intro to Apache Spark
Intro to Apache SparkIntro to Apache Spark
Intro to Apache Spark
 
Spark Kernel Talk - Apache Spark Meetup San Francisco (July 2015)
Spark Kernel Talk - Apache Spark Meetup San Francisco (July 2015)Spark Kernel Talk - Apache Spark Meetup San Francisco (July 2015)
Spark Kernel Talk - Apache Spark Meetup San Francisco (July 2015)
 
Sa introduction to big data pipelining with cassandra & spark west mins...
Sa introduction to big data pipelining with cassandra & spark   west mins...Sa introduction to big data pipelining with cassandra & spark   west mins...
Sa introduction to big data pipelining with cassandra & spark west mins...
 
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
Streaming Big Data with Spark, Kafka, Cassandra, Akka & Scala (from webinar)
 
Using Spark, Kafka, Cassandra and Akka on Mesos for Real-Time Personalization
Using Spark, Kafka, Cassandra and Akka on Mesos for Real-Time PersonalizationUsing Spark, Kafka, Cassandra and Akka on Mesos for Real-Time Personalization
Using Spark, Kafka, Cassandra and Akka on Mesos for Real-Time Personalization
 
Data Science lifecycle with Apache Zeppelin and Spark by Moonsoo Lee
Data Science lifecycle with Apache Zeppelin and Spark by Moonsoo LeeData Science lifecycle with Apache Zeppelin and Spark by Moonsoo Lee
Data Science lifecycle with Apache Zeppelin and Spark by Moonsoo Lee
 
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...
 
Data processing platforms architectures with Spark, Mesos, Akka, Cassandra an...
Data processing platforms architectures with Spark, Mesos, Akka, Cassandra an...Data processing platforms architectures with Spark, Mesos, Akka, Cassandra an...
Data processing platforms architectures with Spark, Mesos, Akka, Cassandra an...
 
Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015Akka in Production - ScalaDays 2015
Akka in Production - ScalaDays 2015
 
Deep Recurrent Neural Networks for Sequence Learning in Spark by Yves Mabiala
Deep Recurrent Neural Networks for Sequence Learning in Spark by Yves MabialaDeep Recurrent Neural Networks for Sequence Learning in Spark by Yves Mabiala
Deep Recurrent Neural Networks for Sequence Learning in Spark by Yves Mabiala
 
Consumer offset management in Kafka
Consumer offset management in KafkaConsumer offset management in Kafka
Consumer offset management in Kafka
 
Spark streaming: Best Practices
Spark streaming: Best PracticesSpark streaming: Best Practices
Spark streaming: Best Practices
 

Similaire à Four Things to Know About Reliable Spark Streaming with Typesafe and 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
 
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...DataWorks Summit
 
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
 
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
 
Productionizing your Streaming Jobs
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming JobsDatabricks
 
Deploying Apache Flume to enable low-latency analytics
Deploying Apache Flume to enable low-latency analyticsDeploying Apache Flume to enable low-latency analytics
Deploying Apache Flume to enable low-latency analyticsDataWorks Summit
 
Spark Study Notes
Spark Study NotesSpark Study Notes
Spark Study NotesRichard Kuo
 
Learning spark ch10 - Spark Streaming
Learning spark ch10 - Spark StreamingLearning spark ch10 - Spark Streaming
Learning spark ch10 - Spark Streamingphanleson
 
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...Lightbend
 
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
 
Flink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseFlink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseKostas Tzoumas
 
R the unsung hero of Big Data
R the unsung hero of Big DataR the unsung hero of Big Data
R the unsung hero of Big DataDhafer Malouche
 
What's New in Apache Spark 2.3 & Why Should You Care
What's New in Apache Spark 2.3 & Why Should You CareWhat's New in Apache Spark 2.3 & Why Should You Care
What's New in Apache Spark 2.3 & Why Should You CareDatabricks
 
Spark Streaming & Kafka-The Future of Stream Processing
Spark Streaming & Kafka-The Future of Stream ProcessingSpark Streaming & Kafka-The Future of Stream Processing
Spark Streaming & Kafka-The Future of Stream ProcessingJack Gudenkauf
 
Spark Streaming& Kafka-The Future of Stream Processing by Hari Shreedharan of...
Spark Streaming& Kafka-The Future of Stream Processing by Hari Shreedharan of...Spark Streaming& Kafka-The Future of Stream Processing by Hari Shreedharan of...
Spark Streaming& Kafka-The Future of Stream Processing by Hari Shreedharan of...Data Con LA
 
Apache Big Data EU 2016: Building Streaming Applications with Apache Apex
Apache Big Data EU 2016: Building Streaming Applications with Apache ApexApache Big Data EU 2016: Building Streaming Applications with Apache Apex
Apache Big Data EU 2016: Building Streaming Applications with Apache ApexApache Apex
 
xPatterns on Spark, Shark, Mesos, Tachyon
xPatterns on Spark, Shark, Mesos, TachyonxPatterns on Spark, Shark, Mesos, Tachyon
xPatterns on Spark, Shark, Mesos, TachyonClaudiu Barbura
 
Spark (Structured) Streaming vs. Kafka Streams
Spark (Structured) Streaming vs. Kafka StreamsSpark (Structured) Streaming vs. Kafka Streams
Spark (Structured) Streaming vs. Kafka StreamsGuido Schmutz
 

Similaire à Four Things to Know About Reliable Spark Streaming with Typesafe and Databricks (20)

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...
 
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
 
20170126 big data processing
20170126 big data processing20170126 big data processing
20170126 big data processing
 
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
 
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...
 
Productionizing your Streaming Jobs
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming Jobs
 
Deploying Apache Flume to enable low-latency analytics
Deploying Apache Flume to enable low-latency analyticsDeploying Apache Flume to enable low-latency analytics
Deploying Apache Flume to enable low-latency analytics
 
Spark Study Notes
Spark Study NotesSpark Study Notes
Spark Study Notes
 
Learning spark ch10 - Spark Streaming
Learning spark ch10 - Spark StreamingLearning spark ch10 - Spark Streaming
Learning spark ch10 - Spark Streaming
 
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
Lessons Learned From PayPal: Implementing Back-Pressure With Akka Streams And...
 
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...
 
Flink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseFlink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San Jose
 
R the unsung hero of Big Data
R the unsung hero of Big DataR the unsung hero of Big Data
R the unsung hero of Big Data
 
What's New in Apache Spark 2.3 & Why Should You Care
What's New in Apache Spark 2.3 & Why Should You CareWhat's New in Apache Spark 2.3 & Why Should You Care
What's New in Apache Spark 2.3 & Why Should You Care
 
Spark Streaming & Kafka-The Future of Stream Processing
Spark Streaming & Kafka-The Future of Stream ProcessingSpark Streaming & Kafka-The Future of Stream Processing
Spark Streaming & Kafka-The Future of Stream Processing
 
Spark Streaming& Kafka-The Future of Stream Processing by Hari Shreedharan of...
Spark Streaming& Kafka-The Future of Stream Processing by Hari Shreedharan of...Spark Streaming& Kafka-The Future of Stream Processing by Hari Shreedharan of...
Spark Streaming& Kafka-The Future of Stream Processing by Hari Shreedharan of...
 
So you think you can stream.pptx
So you think you can stream.pptxSo you think you can stream.pptx
So you think you can stream.pptx
 
Apache Big Data EU 2016: Building Streaming Applications with Apache Apex
Apache Big Data EU 2016: Building Streaming Applications with Apache ApexApache Big Data EU 2016: Building Streaming Applications with Apache Apex
Apache Big Data EU 2016: Building Streaming Applications with Apache Apex
 
xPatterns on Spark, Shark, Mesos, Tachyon
xPatterns on Spark, Shark, Mesos, TachyonxPatterns on Spark, Shark, Mesos, Tachyon
xPatterns on Spark, Shark, Mesos, Tachyon
 
Spark (Structured) Streaming vs. Kafka Streams
Spark (Structured) Streaming vs. Kafka StreamsSpark (Structured) Streaming vs. Kafka Streams
Spark (Structured) Streaming vs. Kafka Streams
 

Plus de Legacy Typesafe (now Lightbend)

The How and Why of Fast Data Analytics with Apache Spark
The How and Why of Fast Data Analytics with Apache SparkThe How and Why of Fast Data Analytics with Apache Spark
The How and Why of Fast Data Analytics with Apache SparkLegacy Typesafe (now Lightbend)
 
Typesafe Reactive Platform: Monitoring 1.0, Commercial features and more
Typesafe Reactive Platform: Monitoring 1.0, Commercial features and moreTypesafe Reactive Platform: Monitoring 1.0, Commercial features and more
Typesafe Reactive Platform: Monitoring 1.0, Commercial features and moreLegacy Typesafe (now Lightbend)
 
Akka 2.4 plus new commercial features in Typesafe Reactive Platform
Akka 2.4 plus new commercial features in Typesafe Reactive PlatformAkka 2.4 plus new commercial features in Typesafe Reactive Platform
Akka 2.4 plus new commercial features in Typesafe Reactive PlatformLegacy Typesafe (now Lightbend)
 
Reactive Revealed Part 3 of 3: Resiliency, Failures vs Errors, Isolation, Del...
Reactive Revealed Part 3 of 3: Resiliency, Failures vs Errors, Isolation, Del...Reactive Revealed Part 3 of 3: Resiliency, Failures vs Errors, Isolation, Del...
Reactive Revealed Part 3 of 3: Resiliency, Failures vs Errors, Isolation, Del...Legacy Typesafe (now Lightbend)
 
Akka 2.4 plus commercial features in Typesafe Reactive Platform
Akka 2.4 plus commercial features in Typesafe Reactive PlatformAkka 2.4 plus commercial features in Typesafe Reactive Platform
Akka 2.4 plus commercial features in Typesafe Reactive PlatformLegacy Typesafe (now Lightbend)
 
Reactive Revealed Part 2: Scalability, Elasticity and Location Transparency i...
Reactive Revealed Part 2: Scalability, Elasticity and Location Transparency i...Reactive Revealed Part 2: Scalability, Elasticity and Location Transparency i...
Reactive Revealed Part 2: Scalability, Elasticity and Location Transparency i...Legacy Typesafe (now Lightbend)
 
Microservices 101: Exploiting Reality's Constraints with Technology
Microservices 101: Exploiting Reality's Constraints with TechnologyMicroservices 101: Exploiting Reality's Constraints with Technology
Microservices 101: Exploiting Reality's Constraints with TechnologyLegacy Typesafe (now Lightbend)
 
A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0
A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0
A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0Legacy Typesafe (now Lightbend)
 
Modernizing Your Aging Architecture: What Enterprise Architects Need To Know ...
Modernizing Your Aging Architecture: What Enterprise Architects Need To Know ...Modernizing Your Aging Architecture: What Enterprise Architects Need To Know ...
Modernizing Your Aging Architecture: What Enterprise Architects Need To Know ...Legacy Typesafe (now Lightbend)
 
Reactive Streams 1.0.0 and Why You Should Care (webinar)
Reactive Streams 1.0.0 and Why You Should Care (webinar)Reactive Streams 1.0.0 and Why You Should Care (webinar)
Reactive Streams 1.0.0 and Why You Should Care (webinar)Legacy Typesafe (now Lightbend)
 
[Sneak Preview] Apache Spark: Preparing for the next wave of Reactive Big Data
[Sneak Preview] Apache Spark: Preparing for the next wave of Reactive Big Data[Sneak Preview] Apache Spark: Preparing for the next wave of Reactive Big Data
[Sneak Preview] Apache Spark: Preparing for the next wave of Reactive Big DataLegacy Typesafe (now Lightbend)
 

Plus de Legacy Typesafe (now Lightbend) (15)

The How and Why of Fast Data Analytics with Apache Spark
The How and Why of Fast Data Analytics with Apache SparkThe How and Why of Fast Data Analytics with Apache Spark
The How and Why of Fast Data Analytics with Apache Spark
 
Reactive Design Patterns
Reactive Design PatternsReactive Design Patterns
Reactive Design Patterns
 
Revitalizing Aging Architectures with Microservices
Revitalizing Aging Architectures with MicroservicesRevitalizing Aging Architectures with Microservices
Revitalizing Aging Architectures with Microservices
 
Typesafe Reactive Platform: Monitoring 1.0, Commercial features and more
Typesafe Reactive Platform: Monitoring 1.0, Commercial features and moreTypesafe Reactive Platform: Monitoring 1.0, Commercial features and more
Typesafe Reactive Platform: Monitoring 1.0, Commercial features and more
 
Akka 2.4 plus new commercial features in Typesafe Reactive Platform
Akka 2.4 plus new commercial features in Typesafe Reactive PlatformAkka 2.4 plus new commercial features in Typesafe Reactive Platform
Akka 2.4 plus new commercial features in Typesafe Reactive Platform
 
Reactive Revealed Part 3 of 3: Resiliency, Failures vs Errors, Isolation, Del...
Reactive Revealed Part 3 of 3: Resiliency, Failures vs Errors, Isolation, Del...Reactive Revealed Part 3 of 3: Resiliency, Failures vs Errors, Isolation, Del...
Reactive Revealed Part 3 of 3: Resiliency, Failures vs Errors, Isolation, Del...
 
Akka 2.4 plus commercial features in Typesafe Reactive Platform
Akka 2.4 plus commercial features in Typesafe Reactive PlatformAkka 2.4 plus commercial features in Typesafe Reactive Platform
Akka 2.4 plus commercial features in Typesafe Reactive Platform
 
Reactive Revealed Part 2: Scalability, Elasticity and Location Transparency i...
Reactive Revealed Part 2: Scalability, Elasticity and Location Transparency i...Reactive Revealed Part 2: Scalability, Elasticity and Location Transparency i...
Reactive Revealed Part 2: Scalability, Elasticity and Location Transparency i...
 
Microservices 101: Exploiting Reality's Constraints with Technology
Microservices 101: Exploiting Reality's Constraints with TechnologyMicroservices 101: Exploiting Reality's Constraints with Technology
Microservices 101: Exploiting Reality's Constraints with Technology
 
A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0
A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0
A Deeper Look Into Reactive Streams with Akka Streams 1.0 and Slick 3.0
 
Modernizing Your Aging Architecture: What Enterprise Architects Need To Know ...
Modernizing Your Aging Architecture: What Enterprise Architects Need To Know ...Modernizing Your Aging Architecture: What Enterprise Architects Need To Know ...
Modernizing Your Aging Architecture: What Enterprise Architects Need To Know ...
 
Reactive Streams 1.0.0 and Why You Should Care (webinar)
Reactive Streams 1.0.0 and Why You Should Care (webinar)Reactive Streams 1.0.0 and Why You Should Care (webinar)
Reactive Streams 1.0.0 and Why You Should Care (webinar)
 
Going Reactive in Java with Typesafe Reactive Platform
Going Reactive in Java with Typesafe Reactive PlatformGoing Reactive in Java with Typesafe Reactive Platform
Going Reactive in Java with Typesafe Reactive Platform
 
Why Play Framework is fast
Why Play Framework is fastWhy Play Framework is fast
Why Play Framework is fast
 
[Sneak Preview] Apache Spark: Preparing for the next wave of Reactive Big Data
[Sneak Preview] Apache Spark: Preparing for the next wave of Reactive Big Data[Sneak Preview] Apache Spark: Preparing for the next wave of Reactive Big Data
[Sneak Preview] Apache Spark: Preparing for the next wave of Reactive Big Data
 

Dernier

Zer0con 2024 final share short version.pdf
Zer0con 2024 final share short version.pdfZer0con 2024 final share short version.pdf
Zer0con 2024 final share short version.pdfmaor17
 
Large Language Models for Test Case Evolution and Repair
Large Language Models for Test Case Evolution and RepairLarge Language Models for Test Case Evolution and Repair
Large Language Models for Test Case Evolution and RepairLionel Briand
 
Leveraging AI for Mobile App Testing on Real Devices | Applitools + Kobiton
Leveraging AI for Mobile App Testing on Real Devices | Applitools + KobitonLeveraging AI for Mobile App Testing on Real Devices | Applitools + Kobiton
Leveraging AI for Mobile App Testing on Real Devices | Applitools + KobitonApplitools
 
Osi security architecture in network.pptx
Osi security architecture in network.pptxOsi security architecture in network.pptx
Osi security architecture in network.pptxVinzoCenzo
 
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...OnePlan Solutions
 
SAM Training Session - How to use EXCEL ?
SAM Training Session - How to use EXCEL ?SAM Training Session - How to use EXCEL ?
SAM Training Session - How to use EXCEL ?Alexandre Beguel
 
[ CNCF Q1 2024 ] Intro to Continuous Profiling and Grafana Pyroscope.pdf
[ CNCF Q1 2024 ] Intro to Continuous Profiling and Grafana Pyroscope.pdf[ CNCF Q1 2024 ] Intro to Continuous Profiling and Grafana Pyroscope.pdf
[ CNCF Q1 2024 ] Intro to Continuous Profiling and Grafana Pyroscope.pdfSteve Caron
 
Ronisha Informatics Private Limited Catalogue
Ronisha Informatics Private Limited CatalogueRonisha Informatics Private Limited Catalogue
Ronisha Informatics Private Limited Catalogueitservices996
 
Enhancing Supply Chain Visibility with Cargo Cloud Solutions.pdf
Enhancing Supply Chain Visibility with Cargo Cloud Solutions.pdfEnhancing Supply Chain Visibility with Cargo Cloud Solutions.pdf
Enhancing Supply Chain Visibility with Cargo Cloud Solutions.pdfRTS corp
 
Best Angular 17 Classroom & Online training - Naresh IT
Best Angular 17 Classroom & Online training - Naresh ITBest Angular 17 Classroom & Online training - Naresh IT
Best Angular 17 Classroom & Online training - Naresh ITmanoharjgpsolutions
 
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full RecordingOpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full RecordingShane Coughlan
 
Mastering Project Planning with Microsoft Project 2016.pptx
Mastering Project Planning with Microsoft Project 2016.pptxMastering Project Planning with Microsoft Project 2016.pptx
Mastering Project Planning with Microsoft Project 2016.pptxAS Design & AST.
 
Amazon Bedrock in Action - presentation of the Bedrock's capabilities
Amazon Bedrock in Action - presentation of the Bedrock's capabilitiesAmazon Bedrock in Action - presentation of the Bedrock's capabilities
Amazon Bedrock in Action - presentation of the Bedrock's capabilitiesKrzysztofKkol1
 
Advantages of Cargo Cloud Solutions.pptx
Advantages of Cargo Cloud Solutions.pptxAdvantages of Cargo Cloud Solutions.pptx
Advantages of Cargo Cloud Solutions.pptxRTS corp
 
Introduction to Firebase Workshop Slides
Introduction to Firebase Workshop SlidesIntroduction to Firebase Workshop Slides
Introduction to Firebase Workshop Slidesvaideheekore1
 
Understanding Flamingo - DeepMind's VLM Architecture
Understanding Flamingo - DeepMind's VLM ArchitectureUnderstanding Flamingo - DeepMind's VLM Architecture
Understanding Flamingo - DeepMind's VLM Architecturerahul_net
 
Tech Tuesday Slides - Introduction to Project Management with OnePlan's Work ...
Tech Tuesday Slides - Introduction to Project Management with OnePlan's Work ...Tech Tuesday Slides - Introduction to Project Management with OnePlan's Work ...
Tech Tuesday Slides - Introduction to Project Management with OnePlan's Work ...OnePlan Solutions
 
The Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptx
The Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptxThe Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptx
The Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptxRTS corp
 
2024 DevNexus Patterns for Resiliency: Shuffle shards
2024 DevNexus Patterns for Resiliency: Shuffle shards2024 DevNexus Patterns for Resiliency: Shuffle shards
2024 DevNexus Patterns for Resiliency: Shuffle shardsChristopher Curtin
 
The Ultimate Guide to Performance Testing in Low-Code, No-Code Environments (...
The Ultimate Guide to Performance Testing in Low-Code, No-Code Environments (...The Ultimate Guide to Performance Testing in Low-Code, No-Code Environments (...
The Ultimate Guide to Performance Testing in Low-Code, No-Code Environments (...kalichargn70th171
 

Dernier (20)

Zer0con 2024 final share short version.pdf
Zer0con 2024 final share short version.pdfZer0con 2024 final share short version.pdf
Zer0con 2024 final share short version.pdf
 
Large Language Models for Test Case Evolution and Repair
Large Language Models for Test Case Evolution and RepairLarge Language Models for Test Case Evolution and Repair
Large Language Models for Test Case Evolution and Repair
 
Leveraging AI for Mobile App Testing on Real Devices | Applitools + Kobiton
Leveraging AI for Mobile App Testing on Real Devices | Applitools + KobitonLeveraging AI for Mobile App Testing on Real Devices | Applitools + Kobiton
Leveraging AI for Mobile App Testing on Real Devices | Applitools + Kobiton
 
Osi security architecture in network.pptx
Osi security architecture in network.pptxOsi security architecture in network.pptx
Osi security architecture in network.pptx
 
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
Revolutionizing the Digital Transformation Office - Leveraging OnePlan’s AI a...
 
SAM Training Session - How to use EXCEL ?
SAM Training Session - How to use EXCEL ?SAM Training Session - How to use EXCEL ?
SAM Training Session - How to use EXCEL ?
 
[ CNCF Q1 2024 ] Intro to Continuous Profiling and Grafana Pyroscope.pdf
[ CNCF Q1 2024 ] Intro to Continuous Profiling and Grafana Pyroscope.pdf[ CNCF Q1 2024 ] Intro to Continuous Profiling and Grafana Pyroscope.pdf
[ CNCF Q1 2024 ] Intro to Continuous Profiling and Grafana Pyroscope.pdf
 
Ronisha Informatics Private Limited Catalogue
Ronisha Informatics Private Limited CatalogueRonisha Informatics Private Limited Catalogue
Ronisha Informatics Private Limited Catalogue
 
Enhancing Supply Chain Visibility with Cargo Cloud Solutions.pdf
Enhancing Supply Chain Visibility with Cargo Cloud Solutions.pdfEnhancing Supply Chain Visibility with Cargo Cloud Solutions.pdf
Enhancing Supply Chain Visibility with Cargo Cloud Solutions.pdf
 
Best Angular 17 Classroom & Online training - Naresh IT
Best Angular 17 Classroom & Online training - Naresh ITBest Angular 17 Classroom & Online training - Naresh IT
Best Angular 17 Classroom & Online training - Naresh IT
 
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full RecordingOpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
OpenChain AI Study Group - Europe and Asia Recap - 2024-04-11 - Full Recording
 
Mastering Project Planning with Microsoft Project 2016.pptx
Mastering Project Planning with Microsoft Project 2016.pptxMastering Project Planning with Microsoft Project 2016.pptx
Mastering Project Planning with Microsoft Project 2016.pptx
 
Amazon Bedrock in Action - presentation of the Bedrock's capabilities
Amazon Bedrock in Action - presentation of the Bedrock's capabilitiesAmazon Bedrock in Action - presentation of the Bedrock's capabilities
Amazon Bedrock in Action - presentation of the Bedrock's capabilities
 
Advantages of Cargo Cloud Solutions.pptx
Advantages of Cargo Cloud Solutions.pptxAdvantages of Cargo Cloud Solutions.pptx
Advantages of Cargo Cloud Solutions.pptx
 
Introduction to Firebase Workshop Slides
Introduction to Firebase Workshop SlidesIntroduction to Firebase Workshop Slides
Introduction to Firebase Workshop Slides
 
Understanding Flamingo - DeepMind's VLM Architecture
Understanding Flamingo - DeepMind's VLM ArchitectureUnderstanding Flamingo - DeepMind's VLM Architecture
Understanding Flamingo - DeepMind's VLM Architecture
 
Tech Tuesday Slides - Introduction to Project Management with OnePlan's Work ...
Tech Tuesday Slides - Introduction to Project Management with OnePlan's Work ...Tech Tuesday Slides - Introduction to Project Management with OnePlan's Work ...
Tech Tuesday Slides - Introduction to Project Management with OnePlan's Work ...
 
The Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptx
The Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptxThe Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptx
The Role of IoT and Sensor Technology in Cargo Cloud Solutions.pptx
 
2024 DevNexus Patterns for Resiliency: Shuffle shards
2024 DevNexus Patterns for Resiliency: Shuffle shards2024 DevNexus Patterns for Resiliency: Shuffle shards
2024 DevNexus Patterns for Resiliency: Shuffle shards
 
The Ultimate Guide to Performance Testing in Low-Code, No-Code Environments (...
The Ultimate Guide to Performance Testing in Low-Code, No-Code Environments (...The Ultimate Guide to Performance Testing in Low-Code, No-Code Environments (...
The Ultimate Guide to Performance Testing in Low-Code, No-Code Environments (...
 

Four Things to Know About Reliable Spark Streaming with Typesafe and Databricks

  • 1. Four Things to Know about Reliable Spark Streaming Dean Wampler, Typesafe Tathagata Das, Databricks
  • 2. Agenda for today • The Stream Processing Landscape • How Spark Streaming Works - A Quick Overview • Features in Spark Streaming that Help Prevent Data Loss • Design Tips for Successful Streaming Applications
  • 7. How Spark Streaming Works: A Quick Overview
  • 8. Spark Streaming Scalable, fault-tolerant stream processing system File systems Databases Dashboards Flume Kinesis HDFS/S3 Kafka Twitter High-level API joins, windows, … often 5x less code Fault-tolerant Exactly-once semantics, even for stateful ops Integration Integrates with MLlib, SQL, DataFrames, GraphX
  • 9. Spark Streaming Receivers receive data streams and chop them up into batches Spark processes the batches and pushes out the results 9 data streams receivers batches results
  • 10. Word Count with Kafka val!context!=!new!StreamingContext(conf,!Seconds(1))! val!lines!=!KafkaUtils.createStream(context,!...)! 10 entry point of streaming functionality create DStream from Kafka data
  • 11. Word Count with Kafka val!context!=!new!StreamingContext(conf,!Seconds(1))! val!lines!=!KafkaUtils.createStream(context,!...)! val!words!=!lines.flatMap(_.split("!"))! 11 split lines into words
  • 12. Word Count with Kafka val!context!=!new!StreamingContext(conf,!Seconds(1))! val!lines!=!KafkaUtils.createStream(context,!...)! val!words!=!lines.flatMap(_.split("!"))! val!wordCounts!=!words.map(x!=>!(x,!1))! !!!!!!!!!!!!!!!!!!!!!!.reduceByKey(_!+!_)! wordCounts.print()! context.start()! 12 print some counts on screen count the words start receiving and transforming the data
  • 13. Word Count with Kafka object!WordCount!{! !!def!main(args:!Array[String])!{! !!!!val!context!=!new!StreamingContext(new!SparkConf(),!Seconds(1))! !!!!val!lines!=!KafkaUtils.createStream(context,!...)! !!!!val!words!=!lines.flatMap(_.split("!"))! !!!!val!wordCounts!=!words.map(x!=>!(x,1)).reduceByKey(_!+!_)! !!!!wordCounts.print()! !!!!context.start()! !!!!context.awaitTermination()! !!}! }! 13
  • 14. Features in Spark Streaming that Help Prevent Data Loss
  • 15. 15 A Deeper View of Spark Streaming
  • 16. Any Spark Application 16 Spark Driver User code runs in the driver process YARN / Mesos / Spark Standalone cluster
  • 17. Any Spark Application 17 Spark Driver User code runs in the driver process YARN / Mesos / Spark Standalone cluster Spark Executor Spark Executor Spark Executor Driver launches executors in cluster
  • 18. Any Spark Application 18 Spark Driver User code runs in the driver process YARN / Mesos / Spark Standalone cluster Tasks sent to executors for processing data Spark Executor Spark Executor Spark Executor Driver launches executors in cluster
  • 19. Spark Streaming Application: Receive data 19 Executor Executor Driver runs receivers as long running tasks Receiver Data stream Driver@ object!WordCount!{! !!def!main(args:!Array[String])!{! !!!!val!context!=!new!StreamingContext(...)! !!!!val!lines!=!KafkaUtils.createStream(...)! !!!!val!words!=!lines.flatMap(_.split("!"))! !!!!val!wordCounts!=!words.map(x!=>!(x,1))! !!!!!!!!!!!!!!!!!!!!!!!!!!.reduceByKey(_!+!_)! !!!!wordCounts.print()! !!!!context.start()! !!!!context.awaitTermination()! !!}! }!
  • 20. Spark Streaming Application: Receive data 20 Executor Executor Driver runs receivers as long running tasks Receiver Data stream Driver@ object!WordCount!{! !!def!main(args:!Array[String])!{! !!!!val!context!=!new!StreamingContext(...)! !!!!val!lines!=!KafkaUtils.createStream(...)! !!!!val!words!=!lines.flatMap(_.split("!"))! !!!!val!wordCounts!=!words.map(x!=>!(x,1))! !!!!!!!!!!!!!!!!!!!!!!!!!!.reduceByKey(_!+!_)! !!!!wordCounts.print()! !!!!context.start()! !!!!context.awaitTermination()! !!}! }! Receiver divides stream into blocks and keeps in memory Data Blocks!
  • 21. Spark Streaming Application: Receive data 21 Executor Executor Driver runs receivers as long running tasks Receiver Data stream Driver@ object!WordCount!{! !!def!main(args:!Array[String])!{! !!!!val!context!=!new!StreamingContext(...)! !!!!val!lines!=!KafkaUtils.createStream(...)! !!!!val!words!=!lines.flatMap(_.split("!"))! !!!!val!wordCounts!=!words.map(x!=>!(x,1))! !!!!!!!!!!!!!!!!!!!!!!!!!!.reduceByKey(_!+!_)! !!!!wordCounts.print()! !!!!context.start()! !!!!context.awaitTermination()! !!}! }! Receiver divides stream into blocks and keeps in memory Data Blocks! Blocks also replicated to another executor Data Blocks!
  • 22. Spark Streaming Application: Process data 22 Executor Executor Receiver Data Blocks! Data Blocks! Every batch interval, driver launches tasks to process the blocks Driver@ object!WordCount!{! !!def!main(args:!Array[String])!{! !!!!val!context!=!new!StreamingContext(...)! !!!!val!lines!=!KafkaUtils.createStream(...)! !!!!val!words!=!lines.flatMap(_.split("!"))! !!!!val!wordCounts!=!words.map(x!=>!(x,1))! !!!!!!!!!!!!!!!!!!!!!!!!!!.reduceByKey(_!+!_)! !!!!wordCounts.print()! !!!!context.start()! !!!!context.awaitTermination()! !!}! }!
  • 23. Spark Streaming Application: Process data 23 Executor Executor Receiver Data Blocks! Data Blocks! Data store Every batch interval, driver launches tasks to process the blocks Driver@ object!WordCount!{! !!def!main(args:!Array[String])!{! !!!!val!context!=!new!StreamingContext(...)! !!!!val!lines!=!KafkaUtils.createStream(...)! !!!!val!words!=!lines.flatMap(_.split("!"))! !!!!val!wordCounts!=!words.map(x!=>!(x,1))! !!!!!!!!!!!!!!!!!!!!!!!!!!.reduceByKey(_!+!_)! !!!!wordCounts.print()! !!!!context.start()! !!!!context.awaitTermination()! !!}! }!
  • 25. Failures? Why care? Many streaming applications need zero data loss guarantees despite any kind of failures in the system At least once guarantee – every record processed at least once Exactly once guarantee – every record processed exactly once Different kinds of failures – executor and driver Some failures and guarantee requirements need additional configurations and setups 25
  • 26. Executor Receiver Data Blocks! What if an executor fails? 26 Executor Failed Ex. Receiver Blocks! Blocks! Driver If executor fails, receiver is lost and all blocks are lost
  • 27. Executor Receiver Data Blocks! What if an executor fails? Tasks and receivers restarted by Spark automatically, no config needed 27 Executor Failed Ex. Receiver Blocks! Blocks! Driver If executor fails, receiver is lost and all blocks are lost Receiver Receiver restarted Tasks restarted on block replicas
  • 28. What if the driver fails? 28 Executor Blocks!How do we recover? When the driver fails, all the executors fail All computation, all received blocks are lost Executor Receiver Blocks! Failed Ex. Receiver Blocks! Failed Executor Blocks! Driver Failed Driver
  • 29. Recovering Driver w/ DStream Checkpointing DStream Checkpointing: Periodically save the DAG of DStreams to fault-tolerant storage 29 Executor Blocks! Executor Receiver Blocks! Active Driver Checkpoint info to HDFS / S3
  • 30. Recovering Driver w/ DStream Checkpointing 30 Failed driver can be restarted from checkpoint information Failed Driver Restarted Driver DStream Checkpointing: Periodically save the DAG of DStreams to fault-tolerant storage
  • 31. Recovering Driver w/ DStream Checkpointing 31 Failed driver can be restarted from checkpoint information Failed Driver Restarted Driver New Executor New Executor Receiver New executors launched and receivers restarted DStream Checkpointing: Periodically save the DAG of DStreams to fault-tolerant storage
  • 32. Recovering Driver w/ DStream Checkpointing 1.  Configure automatic driver restart All cluster managers support this 2.  Set a checkpoint directory in a HDFS-compatible file system !streamingContext.checkpoint(hdfsDirectory)! 3.  Slightly restructure of the code to use checkpoints for recovery 32
  • 33. Configurating Automatic Driver Restart Spark Standalone – Use spark-submit with “cluster” mode and “-- supervise” See http://spark.apache.org/docs/latest/spark-standalone.html YARN – Use spark-submit in “cluster” mode See YARN config “yarn.resourcemanager.am.max-attempts” Mesos – Marathon can restart applications or use the “--supervise” flag. 33
  • 34. Restructuring code for Checkpointing 34 val!context!=!new!StreamingContext(...)! val!lines!=!KafkaUtils.createStream(...)! val!words!=!lines.flatMap(...)! ...! context.start()! Create + Setup Start
  • 35. Restructuring code for Checkpointing 35 val!context!=!new!StreamingContext(...)! val!lines!=!KafkaUtils.createStream(...)! val!words!=!lines.flatMap(...)! ...! context.start()! Create + Setup Start def!creatingFunc():!StreamingContext!=!{!! !!!val!context!=!new!StreamingContext(...)!!! !!!val!lines!=!KafkaUtils.createStream(...)! !!!val!words!=!lines.flatMap(...)! !!!...! !!!context.checkpoint(hdfsDir)! }! Put all setup code into a function that returns a new StreamingContext
  • 36. Restructuring code for Checkpointing 36 val!context!=!new!StreamingContext(...)! val!lines!=!KafkaUtils.createStream(...)! val!words!=!lines.flatMap(...)! ...! context.start()! Create + Setup Start def!creatingFunc():!StreamingContext!=!{!! !!!val!context!=!new!StreamingContext(...)!!! !!!val!lines!=!KafkaUtils.createStream(...)! !!!val!words!=!lines.flatMap(...)! !!!...! !!!context.checkpoint(hdfsDir)! }! Put all setup code into a function that returns a new StreamingContext Get context setup from HDFS dir OR create a new one with the function val!context!=! StreamingContext.getOrCreate(! !!hdfsDir,!creatingFunc)! context.start()!
  • 37. Restructuring code for Checkpointing StreamingContext.getOrCreate(): If HDFS directory has checkpoint info recover context from info else call creatingFunc() to create and setup a new context Restarted process can figure out whether to recover using checkpoint info or not 37 def!creatingFunc():!StreamingContext!=!{!! !!!val!context!=!new!StreamingContext(...)!!! !!!val!lines!=!KafkaUtils.createStream(...)! !!!val!words!=!lines.flatMap(...)! !!!...! !!!context.checkpoint(hdfsDir)! }! val!context!=! StreamingContext.getOrCreate(! !!hdfsDir,!creatingFunc)! context.start()!
  • 38. Received blocks lost on Restart! 38 Failed Driver Restarted Driver New Executor New Ex. Receiver No Blocks! In-memory blocks of buffered data are lost on driver restart
  • 39. Recovering data with Write Ahead Logs Write Ahead Log (WAL): Synchronously save received data to fault-tolerant storage 39 Executor Blocks saved to HDFS Executor Receiver Blocks! Active Driver Data stream
  • 40. Recovering data with Write Ahead Logs 40 Failed Driver Restarted Driver New Executor New Ex. Receiver Blocks! Blocks recovered from Write Ahead Log Write Ahead Log (WAL): Synchronously save received data to fault-tolerant storage
  • 41. Recovering data with Write Ahead Logs 1.  Enable checkpointing, logs written in checkpoint directory 3.  Enabled WAL in SparkConf configuration !!!!!sparkConf.set("spark.streaming.receiver.writeAheadLog.enable",!"true")! 3.  Receiver should also be reliable Acknowledge source only after data saved to WAL Unacked data will be replayed from source by restarted receiver 5.  Disable in-memory replication (already replicated by HDFS) !Use StorageLevel.MEMORY_AND_DISK_SER for input DStreams 41
  • 42. RDD Checkpointing •  Stateful stream processing can lead to long RDD lineages •  Long lineage = bad for fault-tolerance, too much recomputation •  RDD checkpointing saves RDD data to the fault-tolerant storage to limit lineage and recomputation •  More: http://spark.apache.org/docs/latest/streaming-programming-guide.html#checkpointing 42
  • 43. Fault-tolerance Semantics 43 Zero data loss = every stage processes each event at least once despite any failure Sources Transforming Sinks Outputting Receiving
  • 44. Fault-tolerance Semantics 44 Sources Transforming Sinks Outputting Receiving Exactly once, as long as received data is not lost Receiving Outputting End-to-end semantics: At-least once
  • 45. Fault-tolerance Semantics 45 Sources Transforming Sinks Outputting Receiving Exactly once, as long as received data is not lost Receiving Outputting Exactly once, if outputs are idempotent or transactional End-to-end semantics: At-least once
  • 46. Fault-tolerance Semantics 46 Sources Transforming Sinks Outputting Receiving Exactly once, as long as received data is not lost At least once, w/ Checkpointing + WAL + Reliable receiversReceiving Outputting Exactly once, if outputs are idempotent or transactional End-to-end semantics: At-least once
  • 47. Fault-tolerance Semantics 47 Exactly once receiving with new Kafka Direct approach Treats Kafka like a replicated log, reads it like a file Does not use receivers No need to create multiple DStreams and union them No need to enable Write Ahead Logs ! @val!directKafkaStream!=!KafkaUtils.createDirectStream(...)! ! https://databricks.com/blog/2015/03/30/improvements-to-kafka-integration-of-spark-streaming.html http://spark.apache.org/docs/latest/streaming-kafka-integration.html Sources Transforming Sinks Outputting Receiving
  • 48. Fault-tolerance Semantics 48 Exactly once receiving with new Kafka Direct approach Sources Transforming Sinks Outputting Receiving Exactly once, as long as received data is not lost Exactly once, if outputs are idempotent or transactional End-to-end semantics: Exactly once!
  • 49. Design Tips for Successful Streaming Applications
  • 50. Areas for consideration • Enhance resilience with additional components. • Mini-batch vs. per-message handling. • Exploit Reactive Streams.
  • 51. • Use Storm, Akka, Samza, etc. for handling individual messages, especially with sub- second latency requirements. • Use Spark Streaming’s mini-batch model for the Lambda architecture and highly-scalable analytics. Mini-batch vs. per-message handling
  • 52. • Consider Kafka or Kinesis for resilient buffering in front of Spark Streaming. •  Buffer for traffic spikes. •  Re-retrieval of data if an RDD partition is lost and must be reconstructed from the source. • Going to store the raw data anyway? •  Do it first, then ingest to Spark from that storage. Enhance Resiliency with Additional Components.
  • 53. • Spark Streaming v1.5 will have support for back pressure to more easily build end-to-end reactive applications Exploit Reactive Streams
  • 54. • Spark Streaming v1.5 will have support for back pressure to more easily build end-to-end reactive applications • Backpressure from consumer to producer: •  Prevents buffer overflows. •  Avoids unnecessary throttling. Exploit Reactive Streams
  • 55. • Spark Streaming v1.4? Buffer with Akka Streams: Exploit Reactive Streams Event Source Akka App Spark Streaming App Event Event Event Event feedback (back pressure)
  • 56. • Spark Streaming v1.4 has a rate limit property: •  spark.streaming.receiver.maxRate •  Consider setting it for long-running streaming apps with a variable input flow rate. • Have a graph of Reactive Streams? Consider using an Akka app to buffer the data fed to Spark Streaming over a socket (until 1.5…). Exploit Reactive Streams
  • 57. Thank you! Dean Wampler, Typesafe Tathagata Das, Databricks
  • 58. Databricks is available as a hosted platform on  AWS with a monthly subscription.  What to do next? Start with a free trial Typesafe now offers certified support for Spark, Mesos & DCOS, read more about it READ MORE
  • 59. REACTIVE PLATFORM
 Full Lifecycle Support for Play, Akka, Scala and Spark Give your project a boost with Reactive Platform: • Monitor Message-Driven Apps • Resolve Network Partitions Decisively • Integrate Easily with Legacy Systems • Eliminate Incompatibility & Security Risks • Protect Apps Against Abuse • Expert Support from Dedicated Product Teams Enjoy learning? See about the availability of on-site training for Scala, Akka, Play and Spark! Learn more about our offersCONTACT US

Notes de l'éditeur

  1. You’ve heard from TD about Streaming internals and how to make your Spark Streaming apps resilient. Let’s expand the view a bit with additional tips.