SlideShare uma empresa Scribd logo
1 de 46
Baixar para ler offline
Productionizing your 

Streaming Jobs
Prakash Chockalingam
@prakash573
About the speaker: Prakash Chockalingam
Prakash is currently a Solutions Architect at
Databricks and focuses on helping customers
building their big data infrastructure based on his
decade-long experience on building large scale
distributed systems and machine learning
infrastructure at companies including Netflix and
Yahoo. Prior to joining Databricks, he was with
Netflix designing and building their
recommendation infrastructure that serves out
millions of recommendations to Netflix users every
day.
2
About the moderator: Denny Lee
Denny Lee is a Technology Evangelist with
Databricks; he is a hands-on data sciences engineer
with more than 15 years of experience developing
internet-scale infrastructure, data platforms, and
distributed systems for both on-premises and cloud.
3
About Databricks
Founded by creators of Spark in 2013
Cloud enterprise data platform
- Managed Spark clusters
- Interactive data science
- Production pipelines
- Data governance, security, …
Agenda
• Introduction to Spark Streaming
• Lifecycle of a Spark streaming app
• Aggregations and best practices
• Operationalization tips
• Key benefits of Spark streaming
What is Spark Streaming?
Spark Streaming
How does it work?
● Receivers receive data streams and chops them in to batches.
● Spark processes the batches and pushes out the results
Word Count
val context = new StreamingContext(conf, Seconds(1))
val lines = context.socketTextStream(...)
Entry point Batch Interval
DStream: represents a
data stream
Word Count
val context = new StreamingContext(conf, Seconds(1))
val lines = context.socketTextStream(...)
val words = lines.flatMap(_.split(“ “))
Transformations: transform
data to create new DStreams
Word Count
val context = new StreamingContext(conf, Seconds(1))
val lines = context.socketTextStream(...)
val words = lines.flatMap(_.split(“ “))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_+_)
wordCounts.print()
context.start() Print the DStream contents on screen
Start the streaming job
Lifecycle of a streaming app
Execution in any Spark Application
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
Execution in Spark Streaming: Receiving data
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
Execution in Spark Streaming: Processing data
Executor
Executor
Receiver
Data Blocks
Data Blocks
results
results
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()
}
}
End-to-end view
17
t1 = ssc.socketStream(“…”)
t2 = ssc.socketStream(“…”)
t = t1.union(t2).map(…)
t.saveAsHadoopFiles(…)
t.map(…).foreach(…)

t.filter(…).foreach(…)

T
U
M
T
M F
FE
FE FE
B
U
M
B
M F
Input DStreams
Output
operations
RDD Actions /
Spark Jobs
BlockRDDs
DStreamGraph
DAG of RDDs
every interval
DAG of stages
every interval
Stage 1
Stage 2
Stage 3
Streaming app
Tasks
every interval
B
U
M
B
M F
B
U
M
B
M F
Stage 1
Stage 2
Stage 3
Stage 1
Stage 2
Stage 3
Spark Streaming
JobScheduler + JobGenerator
Spark
DAGScheduler
Spark
TaskScheduler
Executors
YOU write
this
Aggregations
Word count over a time window
val wordCounts = wordStream.reduceByKeyAndWindow((x:
Int, y:Int) => x+y, windowSize, slidingInterval)
Parent
DStream
window size
sliding
interval
Reduces over a time window
Word count over a time window
Scenario: Word count for the last 30 minutes
How to optimize for good performance?
● Increase batch interval, if possible
● Incremental aggregations with inverse reduce function
val wordCounts = wordStream.reduceByKeyAndWindow(
(x: Int, y:Int) => x+y, (x: Int, y: Int) =>
x-y, windowSize, slidingInterval)
● Checkpointing
wordStream.checkpoint(checkpointInterval)
Stateful: Global Aggregations
Scenario: Maintain a global state based on the input events coming
in. Ex: Word count from beginning of time.
updateStateByKey (Spark 1.5 and before)
● Performance is proportional to the size of the state.
mapWithState (Spark 1.6+)
● Performance is proportional to the size of the batch.
Stateful: Global Aggregations
Stateful: Global Aggregations
Key features of mapWithState:
● An initial state - Read from somewhere as a RDD
● # of partitions for the state - If you have a good estimate of the size of the state,
you can specify the # of partitions.
● Partitioner - Default: Hash partitioner. If you have a good understanding of the
key space, then you can provide a custom partitioner
● Timeout - Keys whose values are not updated within the specified timeout
period will be removed from the state.
Stateful: Global Aggregations (Word count)
val stateSpec = StateSpec.function(updateState _)
.initialState(initialRDD)
.numPartitions(100)
.partitioner(MyPartitioner())
.timeout(Minutes(120))
val wordCountState = wordStream.mapWithState(stateSpec)
Stateful: Global Aggregations (Word count)
def updateState(batchTime: Time,
key: String,
value: Option[Int],
state: State[Long])
: Option[(String, Long)]
Current batch time
A Word in the input stream
Current value (= 1)
Counts so far for the word
The word and its new count
Operationalization
Checkpoint
Two types of checkpointing:
● Checkpointing Data
● Checkpointing Metadata
Checkpoint Data
● Checkpointing DStreams
• Primarily needed to cut long lineage on past batches (updateStateByKey/
reduceByKeyAndWindow).
• Example: wordStream.checkpoint(checkpointInterval)
Checkpoint Metadata
● Checkpointing Metadata
• All the configuration, DStream operations and incomplete batches are
checkpointed.
• Required for failure recovery if the driver process crashes.
• Example: streamingContext.checkpoint(directory)
Achieving good throughput
context.socketStream(...)
.map(...)
.filter(...)
.saveAsHadoopFile(...)
Problem: There will be 1 receiver which receives all the data and stores
it in its executor and all the processing happens on that executor.
Adding more nodes doesn’t help.
Achieving good throughput
Solution: Increase the # of receivers and union them.
● Each receiver is run in 1 executor. Having 5 receivers will ensure
that the data gets received in parallel in 5 executors.
● Data gets distributed in 5 executors. So all the subsequent Spark
map/filter operations will be distributed
val numStreams = 5
val inputStreams = (1 to numStreams).map(i =>
context.socketStream(...))
val fullStream = context.union(inputStreams)
fullStream.map(...).filter(...).saveAsHadoopFile(...)
Achieving good throughput
● In the case of direct receivers (like Kafka), set the appropriate # of
partitions in Kafka.
● Each kafka paratition gets mapped to a Spark partition.
● More partitions in Kafka = More parallelism in Spark
Achieving good throughput
● Provide the right # of partitions based on your cluster size for
operations causing shuffles.
words.map(x => (x, 1)).reduceByKey(_+_, 100)
# of partitions
Debugging a Streaming application
Streaming tab in Spark UI
Debugging a Streaming application
Processing Time
● Make sure that the processing time < batch interval
Debugging a Streaming application
Debugging a Streaming application
Batch Details Page:
● Input to the batch
● Jobs that were run as part of the processing for the batch
Debugging a Streaming application
Job Details Page
● DAG Visualization
● Stages of a Spark job
Debugging a Streaming application
Task Details Page
Ensure that the tasks are executed on multiple executors (nodes) in your cluster to have enough parallelism while processing.
If you have a single receiver, sometimes only one executor might be doing all the work though you have more than one executor
in your cluster.
Key benefits of Spark streaming
Dynamic Load Balancing
Fast failure and Straggler recovery
Combine Batch and Stream Processing
Join data streams with static data sets
val dataset = sparkContext.hadoopFile(“file”)
…
kafkaStream.transform{ batchRdd =>
batchRdd.join(dataset).filter(...)
}
Combine ML and Stream Processing
Learn models offline, apply them online
val model = KMeans.train(dataset, …)
kakfaStream.map { event =>
model.predict(event.feature)
}
Combine SQL and Stream Processing
inputStream.foreachRDD{ rdd =>
val df = SQLContext.createDataframe(rdd)
df.select(...).where(...).groupBy(...)
}
Thank you.

Mais conteúdo relacionado

Mais procurados

A look ahead at spark 2.0
A look ahead at spark 2.0 A look ahead at spark 2.0
A look ahead at spark 2.0 Databricks
 
Deep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0’s OptimizerDeep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0’s OptimizerDatabricks
 
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
 
Data Source API in Spark
Data Source API in SparkData Source API in Spark
Data Source API in SparkDatabricks
 
Unified Big Data Processing with Apache Spark
Unified Big Data Processing with Apache SparkUnified Big Data Processing with Apache Spark
Unified Big Data Processing with Apache SparkC4Media
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkDatabricks
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkPatrick Wendell
 
Real Time Data Processing Using Spark Streaming
Real Time Data Processing Using Spark StreamingReal Time Data Processing Using Spark Streaming
Real Time Data Processing Using Spark StreamingHari Shreedharan
 
Apache Spark RDDs
Apache Spark RDDsApache Spark RDDs
Apache Spark RDDsDean Chen
 
Apache Spark for Library Developers with William Benton and Erik Erlandson
 Apache Spark for Library Developers with William Benton and Erik Erlandson Apache Spark for Library Developers with William Benton and Erik Erlandson
Apache Spark for Library Developers with William Benton and Erik ErlandsonDatabricks
 
From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...
From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...
From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...Spark Summit
 
SparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsSparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsDatabricks
 
Apache® Spark™ 1.6 presented by Databricks co-founder Patrick Wendell
Apache® Spark™ 1.6 presented by Databricks co-founder Patrick WendellApache® Spark™ 1.6 presented by Databricks co-founder Patrick Wendell
Apache® Spark™ 1.6 presented by Databricks co-founder Patrick WendellDatabricks
 
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
 
Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...
Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...
Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...Databricks
 
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...DataWorks Summit
 
Introduction to Apache Spark Developer Training
Introduction to Apache Spark Developer TrainingIntroduction to Apache Spark Developer Training
Introduction to Apache Spark Developer TrainingCloudera, Inc.
 

Mais procurados (20)

A look ahead at spark 2.0
A look ahead at spark 2.0 A look ahead at spark 2.0
A look ahead at spark 2.0
 
Deep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0’s OptimizerDeep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
Deep Dive Into Catalyst: Apache Spark 2.0’s Optimizer
 
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
 
Spark streaming: Best Practices
Spark streaming: Best PracticesSpark streaming: Best Practices
Spark streaming: Best Practices
 
Data Source API in Spark
Data Source API in SparkData Source API in Spark
Data Source API in Spark
 
Unified Big Data Processing with Apache Spark
Unified Big Data Processing with Apache SparkUnified Big Data Processing with Apache Spark
Unified Big Data Processing with Apache Spark
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
 
Spark etl
Spark etlSpark etl
Spark etl
 
Real Time Data Processing Using Spark Streaming
Real Time Data Processing Using Spark StreamingReal Time Data Processing Using Spark Streaming
Real Time Data Processing Using Spark Streaming
 
Apache Spark RDDs
Apache Spark RDDsApache Spark RDDs
Apache Spark RDDs
 
Apache Spark for Library Developers with William Benton and Erik Erlandson
 Apache Spark for Library Developers with William Benton and Erik Erlandson Apache Spark for Library Developers with William Benton and Erik Erlandson
Apache Spark for Library Developers with William Benton and Erik Erlandson
 
From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...
From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...
From DataFrames to Tungsten: A Peek into Spark's Future-(Reynold Xin, Databri...
 
Apache Spark
Apache Spark Apache Spark
Apache Spark
 
SparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsSparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDs
 
Apache® Spark™ 1.6 presented by Databricks co-founder Patrick Wendell
Apache® Spark™ 1.6 presented by Databricks co-founder Patrick WendellApache® Spark™ 1.6 presented by Databricks co-founder Patrick Wendell
Apache® Spark™ 1.6 presented by Databricks co-founder Patrick Wendell
 
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...
 
Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...
Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...
Spark Summit EU 2015: Spark DataFrames: Simple and Fast Analysis of Structure...
 
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
 
Introduction to Apache Spark Developer Training
Introduction to Apache Spark Developer TrainingIntroduction to Apache Spark Developer Training
Introduction to Apache Spark Developer Training
 

Semelhante a Productionizing your Streaming Jobs

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
 
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...Inhacking
 
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...Аліна Шепшелей
 
Meet Up - Spark Stream Processing + Kafka
Meet Up - Spark Stream Processing + KafkaMeet Up - Spark Stream Processing + Kafka
Meet Up - Spark Stream Processing + KafkaKnoldus Inc.
 
Spark what's new what's coming
Spark what's new what's comingSpark what's new what's coming
Spark what's new what's comingDatabricks
 
Spark Streaming, Machine Learning and meetup.com streaming API.
Spark Streaming, Machine Learning and  meetup.com streaming API.Spark Streaming, Machine Learning and  meetup.com streaming API.
Spark Streaming, Machine Learning and meetup.com streaming API.Sergey Zelvenskiy
 
Jump Start into Apache® Spark™ and Databricks
Jump Start into Apache® Spark™ and DatabricksJump Start into Apache® Spark™ and Databricks
Jump Start into Apache® Spark™ and DatabricksDatabricks
 
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
 
Distributed Real-Time Stream Processing: Why and How 2.0
Distributed Real-Time Stream Processing:  Why and How 2.0Distributed Real-Time Stream Processing:  Why and How 2.0
Distributed Real-Time Stream Processing: Why and How 2.0Petr Zapletal
 
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)
 
AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)Paul Chao
 
Jump Start with Apache Spark 2.0 on Databricks
Jump Start with Apache Spark 2.0 on DatabricksJump Start with Apache Spark 2.0 on Databricks
Jump Start with Apache Spark 2.0 on DatabricksDatabricks
 
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
 
Data processing platforms with SMACK: Spark and Mesos internals
Data processing platforms with SMACK:  Spark and Mesos internalsData processing platforms with SMACK:  Spark and Mesos internals
Data processing platforms with SMACK: Spark and Mesos internalsAnton Kirillov
 
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
 
Spark 计算模型
Spark 计算模型Spark 计算模型
Spark 计算模型wang xing
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache SparkDatio Big Data
 

Semelhante a Productionizing your Streaming Jobs (20)

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
 
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...
 
20170126 big data processing
20170126 big data processing20170126 big data processing
20170126 big data processing
 
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D...
 
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi...
 
Meet Up - Spark Stream Processing + Kafka
Meet Up - Spark Stream Processing + KafkaMeet Up - Spark Stream Processing + Kafka
Meet Up - Spark Stream Processing + Kafka
 
Spark what's new what's coming
Spark what's new what's comingSpark what's new what's coming
Spark what's new what's coming
 
Data Pipeline at Tapad
Data Pipeline at TapadData Pipeline at Tapad
Data Pipeline at Tapad
 
Spark Streaming, Machine Learning and meetup.com streaming API.
Spark Streaming, Machine Learning and  meetup.com streaming API.Spark Streaming, Machine Learning and  meetup.com streaming API.
Spark Streaming, Machine Learning and meetup.com streaming API.
 
Jump Start into Apache® Spark™ and Databricks
Jump Start into Apache® Spark™ and DatabricksJump Start into Apache® Spark™ and Databricks
Jump Start into Apache® Spark™ and Databricks
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
 
Distributed Real-Time Stream Processing: Why and How 2.0
Distributed Real-Time Stream Processing:  Why and How 2.0Distributed Real-Time Stream Processing:  Why and How 2.0
Distributed Real-Time Stream Processing: Why and How 2.0
 
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
 
AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)
 
Jump Start with Apache Spark 2.0 on Databricks
Jump Start with Apache Spark 2.0 on DatabricksJump Start with Apache Spark 2.0 on Databricks
Jump Start with Apache Spark 2.0 on Databricks
 
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 ...
 
Data processing platforms with SMACK: Spark and Mesos internals
Data processing platforms with SMACK:  Spark and Mesos internalsData processing platforms with SMACK:  Spark and Mesos internals
Data processing platforms with SMACK: Spark and Mesos internals
 
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...
 
Spark 计算模型
Spark 计算模型Spark 计算模型
Spark 计算模型
 
Introduction to Apache Spark
Introduction to Apache SparkIntroduction to Apache Spark
Introduction to Apache Spark
 

Mais de Databricks

DW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDatabricks
 
Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Databricks
 
Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Databricks
 
Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Databricks
 
Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Databricks
 
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of HadoopDatabricks
 
Democratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDemocratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDatabricks
 
Learn to Use Databricks for Data Science
Learn to Use Databricks for Data ScienceLearn to Use Databricks for Data Science
Learn to Use Databricks for Data ScienceDatabricks
 
Why APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML MonitoringWhy APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML MonitoringDatabricks
 
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch FixThe Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch FixDatabricks
 
Stage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationStage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationDatabricks
 
Simplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchSimplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchDatabricks
 
Scaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on KubernetesScaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on KubernetesDatabricks
 
Scaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark PipelinesScaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark PipelinesDatabricks
 
Sawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature AggregationsSawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature AggregationsDatabricks
 
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen SinkRedis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen SinkDatabricks
 
Re-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and SparkRe-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and SparkDatabricks
 
Raven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesRaven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesDatabricks
 
Processing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkProcessing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkDatabricks
 
Massive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta LakeMassive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta LakeDatabricks
 

Mais de Databricks (20)

DW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptx
 
Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1
 
Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2
 
Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2Data Lakehouse Symposium | Day 2
Data Lakehouse Symposium | Day 2
 
Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4
 
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop
 
Democratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDemocratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized Platform
 
Learn to Use Databricks for Data Science
Learn to Use Databricks for Data ScienceLearn to Use Databricks for Data Science
Learn to Use Databricks for Data Science
 
Why APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML MonitoringWhy APM Is Not the Same As ML Monitoring
Why APM Is Not the Same As ML Monitoring
 
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch FixThe Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix
 
Stage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI IntegrationStage Level Scheduling Improving Big Data and AI Integration
Stage Level Scheduling Improving Big Data and AI Integration
 
Simplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorchSimplify Data Conversion from Spark to TensorFlow and PyTorch
Simplify Data Conversion from Spark to TensorFlow and PyTorch
 
Scaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on KubernetesScaling your Data Pipelines with Apache Spark on Kubernetes
Scaling your Data Pipelines with Apache Spark on Kubernetes
 
Scaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark PipelinesScaling and Unifying SciKit Learn and Apache Spark Pipelines
Scaling and Unifying SciKit Learn and Apache Spark Pipelines
 
Sawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature AggregationsSawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature Aggregations
 
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen SinkRedis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink
 
Re-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and SparkRe-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and Spark
 
Raven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction QueriesRaven: End-to-end Optimization of ML Prediction Queries
Raven: End-to-end Optimization of ML Prediction Queries
 
Processing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache SparkProcessing Large Datasets for ADAS Applications using Apache Spark
Processing Large Datasets for ADAS Applications using Apache Spark
 
Massive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta LakeMassive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta Lake
 

Último

Enhancing Worker Digital Experience: A Hands-on Workshop for Partners
Enhancing Worker Digital Experience: A Hands-on Workshop for PartnersEnhancing Worker Digital Experience: A Hands-on Workshop for Partners
Enhancing Worker Digital Experience: A Hands-on Workshop for PartnersThousandEyes
 
A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)Gabriella Davis
 
[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdf[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdfhans926745
 
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...Drew Madelung
 
Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...
Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...
Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...shyamraj55
 
Finology Group – Insurtech Innovation Award 2024
Finology Group – Insurtech Innovation Award 2024Finology Group – Insurtech Innovation Award 2024
Finology Group – Insurtech Innovation Award 2024The Digital Insurer
 
The Codex of Business Writing Software for Real-World Solutions 2.pptx
The Codex of Business Writing Software for Real-World Solutions 2.pptxThe Codex of Business Writing Software for Real-World Solutions 2.pptx
The Codex of Business Writing Software for Real-World Solutions 2.pptxMalak Abu Hammad
 
Understanding the Laravel MVC Architecture
Understanding the Laravel MVC ArchitectureUnderstanding the Laravel MVC Architecture
Understanding the Laravel MVC ArchitecturePixlogix Infotech
 
Presentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreterPresentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreternaman860154
 
GenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day PresentationGenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day PresentationMichael W. Hawkins
 
Slack Application Development 101 Slides
Slack Application Development 101 SlidesSlack Application Development 101 Slides
Slack Application Development 101 Slidespraypatel2
 
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationFrom Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationSafe Software
 
My Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationMy Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationRidwan Fadjar
 
08448380779 Call Girls In Diplomatic Enclave Women Seeking Men
08448380779 Call Girls In Diplomatic Enclave Women Seeking Men08448380779 Call Girls In Diplomatic Enclave Women Seeking Men
08448380779 Call Girls In Diplomatic Enclave Women Seeking MenDelhi Call girls
 
Maximizing Board Effectiveness 2024 Webinar.pptx
Maximizing Board Effectiveness 2024 Webinar.pptxMaximizing Board Effectiveness 2024 Webinar.pptx
Maximizing Board Effectiveness 2024 Webinar.pptxOnBoard
 
Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 365
Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 365Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 365
Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 3652toLead Limited
 
Swan(sea) Song – personal research during my six years at Swansea ... and bey...
Swan(sea) Song – personal research during my six years at Swansea ... and bey...Swan(sea) Song – personal research during my six years at Swansea ... and bey...
Swan(sea) Song – personal research during my six years at Swansea ... and bey...Alan Dix
 
The 7 Things I Know About Cyber Security After 25 Years | April 2024
The 7 Things I Know About Cyber Security After 25 Years | April 2024The 7 Things I Know About Cyber Security After 25 Years | April 2024
The 7 Things I Know About Cyber Security After 25 Years | April 2024Rafal Los
 
08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking Men08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking MenDelhi Call girls
 
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure serviceWhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure servicePooja Nehwal
 

Último (20)

Enhancing Worker Digital Experience: A Hands-on Workshop for Partners
Enhancing Worker Digital Experience: A Hands-on Workshop for PartnersEnhancing Worker Digital Experience: A Hands-on Workshop for Partners
Enhancing Worker Digital Experience: A Hands-on Workshop for Partners
 
A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)
 
[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdf[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdf
 
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
 
Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...
Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...
Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...
 
Finology Group – Insurtech Innovation Award 2024
Finology Group – Insurtech Innovation Award 2024Finology Group – Insurtech Innovation Award 2024
Finology Group – Insurtech Innovation Award 2024
 
The Codex of Business Writing Software for Real-World Solutions 2.pptx
The Codex of Business Writing Software for Real-World Solutions 2.pptxThe Codex of Business Writing Software for Real-World Solutions 2.pptx
The Codex of Business Writing Software for Real-World Solutions 2.pptx
 
Understanding the Laravel MVC Architecture
Understanding the Laravel MVC ArchitectureUnderstanding the Laravel MVC Architecture
Understanding the Laravel MVC Architecture
 
Presentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreterPresentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreter
 
GenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day PresentationGenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day Presentation
 
Slack Application Development 101 Slides
Slack Application Development 101 SlidesSlack Application Development 101 Slides
Slack Application Development 101 Slides
 
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationFrom Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
 
My Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationMy Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 Presentation
 
08448380779 Call Girls In Diplomatic Enclave Women Seeking Men
08448380779 Call Girls In Diplomatic Enclave Women Seeking Men08448380779 Call Girls In Diplomatic Enclave Women Seeking Men
08448380779 Call Girls In Diplomatic Enclave Women Seeking Men
 
Maximizing Board Effectiveness 2024 Webinar.pptx
Maximizing Board Effectiveness 2024 Webinar.pptxMaximizing Board Effectiveness 2024 Webinar.pptx
Maximizing Board Effectiveness 2024 Webinar.pptx
 
Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 365
Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 365Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 365
Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 365
 
Swan(sea) Song – personal research during my six years at Swansea ... and bey...
Swan(sea) Song – personal research during my six years at Swansea ... and bey...Swan(sea) Song – personal research during my six years at Swansea ... and bey...
Swan(sea) Song – personal research during my six years at Swansea ... and bey...
 
The 7 Things I Know About Cyber Security After 25 Years | April 2024
The 7 Things I Know About Cyber Security After 25 Years | April 2024The 7 Things I Know About Cyber Security After 25 Years | April 2024
The 7 Things I Know About Cyber Security After 25 Years | April 2024
 
08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking Men08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking Men
 
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure serviceWhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
 

Productionizing your Streaming Jobs

  • 1. Productionizing your 
 Streaming Jobs Prakash Chockalingam @prakash573
  • 2. About the speaker: Prakash Chockalingam Prakash is currently a Solutions Architect at Databricks and focuses on helping customers building their big data infrastructure based on his decade-long experience on building large scale distributed systems and machine learning infrastructure at companies including Netflix and Yahoo. Prior to joining Databricks, he was with Netflix designing and building their recommendation infrastructure that serves out millions of recommendations to Netflix users every day. 2
  • 3. About the moderator: Denny Lee Denny Lee is a Technology Evangelist with Databricks; he is a hands-on data sciences engineer with more than 15 years of experience developing internet-scale infrastructure, data platforms, and distributed systems for both on-premises and cloud. 3
  • 4. About Databricks Founded by creators of Spark in 2013 Cloud enterprise data platform - Managed Spark clusters - Interactive data science - Production pipelines - Data governance, security, …
  • 5. Agenda • Introduction to Spark Streaming • Lifecycle of a Spark streaming app • Aggregations and best practices • Operationalization tips • Key benefits of Spark streaming
  • 6. What is Spark Streaming?
  • 8.
  • 9. How does it work? ● Receivers receive data streams and chops them in to batches. ● Spark processes the batches and pushes out the results
  • 10. Word Count val context = new StreamingContext(conf, Seconds(1)) val lines = context.socketTextStream(...) Entry point Batch Interval DStream: represents a data stream
  • 11. Word Count val context = new StreamingContext(conf, Seconds(1)) val lines = context.socketTextStream(...) val words = lines.flatMap(_.split(“ “)) Transformations: transform data to create new DStreams
  • 12. Word Count val context = new StreamingContext(conf, Seconds(1)) val lines = context.socketTextStream(...) val words = lines.flatMap(_.split(“ “)) val wordCounts = words.map(x => (x, 1)).reduceByKey(_+_) wordCounts.print() context.start() Print the DStream contents on screen Start the streaming job
  • 13. Lifecycle of a streaming app
  • 14. Execution in any Spark Application 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
  • 15. Execution in Spark Streaming: Receiving data 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
  • 16. Execution in Spark Streaming: Processing data Executor Executor Receiver Data Blocks Data Blocks results results 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() } }
  • 17. End-to-end view 17 t1 = ssc.socketStream(“…”) t2 = ssc.socketStream(“…”) t = t1.union(t2).map(…) t.saveAsHadoopFiles(…) t.map(…).foreach(…)
 t.filter(…).foreach(…)
 T U M T M F FE FE FE B U M B M F Input DStreams Output operations RDD Actions / Spark Jobs BlockRDDs DStreamGraph DAG of RDDs every interval DAG of stages every interval Stage 1 Stage 2 Stage 3 Streaming app Tasks every interval B U M B M F B U M B M F Stage 1 Stage 2 Stage 3 Stage 1 Stage 2 Stage 3 Spark Streaming JobScheduler + JobGenerator Spark DAGScheduler Spark TaskScheduler Executors YOU write this
  • 19. Word count over a time window val wordCounts = wordStream.reduceByKeyAndWindow((x: Int, y:Int) => x+y, windowSize, slidingInterval) Parent DStream window size sliding interval Reduces over a time window
  • 20. Word count over a time window Scenario: Word count for the last 30 minutes How to optimize for good performance? ● Increase batch interval, if possible ● Incremental aggregations with inverse reduce function val wordCounts = wordStream.reduceByKeyAndWindow( (x: Int, y:Int) => x+y, (x: Int, y: Int) => x-y, windowSize, slidingInterval) ● Checkpointing wordStream.checkpoint(checkpointInterval)
  • 21. Stateful: Global Aggregations Scenario: Maintain a global state based on the input events coming in. Ex: Word count from beginning of time. updateStateByKey (Spark 1.5 and before) ● Performance is proportional to the size of the state. mapWithState (Spark 1.6+) ● Performance is proportional to the size of the batch.
  • 23. Stateful: Global Aggregations Key features of mapWithState: ● An initial state - Read from somewhere as a RDD ● # of partitions for the state - If you have a good estimate of the size of the state, you can specify the # of partitions. ● Partitioner - Default: Hash partitioner. If you have a good understanding of the key space, then you can provide a custom partitioner ● Timeout - Keys whose values are not updated within the specified timeout period will be removed from the state.
  • 24. Stateful: Global Aggregations (Word count) val stateSpec = StateSpec.function(updateState _) .initialState(initialRDD) .numPartitions(100) .partitioner(MyPartitioner()) .timeout(Minutes(120)) val wordCountState = wordStream.mapWithState(stateSpec)
  • 25. Stateful: Global Aggregations (Word count) def updateState(batchTime: Time, key: String, value: Option[Int], state: State[Long]) : Option[(String, Long)] Current batch time A Word in the input stream Current value (= 1) Counts so far for the word The word and its new count
  • 27. Checkpoint Two types of checkpointing: ● Checkpointing Data ● Checkpointing Metadata
  • 28. Checkpoint Data ● Checkpointing DStreams • Primarily needed to cut long lineage on past batches (updateStateByKey/ reduceByKeyAndWindow). • Example: wordStream.checkpoint(checkpointInterval)
  • 29. Checkpoint Metadata ● Checkpointing Metadata • All the configuration, DStream operations and incomplete batches are checkpointed. • Required for failure recovery if the driver process crashes. • Example: streamingContext.checkpoint(directory)
  • 30. Achieving good throughput context.socketStream(...) .map(...) .filter(...) .saveAsHadoopFile(...) Problem: There will be 1 receiver which receives all the data and stores it in its executor and all the processing happens on that executor. Adding more nodes doesn’t help.
  • 31. Achieving good throughput Solution: Increase the # of receivers and union them. ● Each receiver is run in 1 executor. Having 5 receivers will ensure that the data gets received in parallel in 5 executors. ● Data gets distributed in 5 executors. So all the subsequent Spark map/filter operations will be distributed val numStreams = 5 val inputStreams = (1 to numStreams).map(i => context.socketStream(...)) val fullStream = context.union(inputStreams) fullStream.map(...).filter(...).saveAsHadoopFile(...)
  • 32. Achieving good throughput ● In the case of direct receivers (like Kafka), set the appropriate # of partitions in Kafka. ● Each kafka paratition gets mapped to a Spark partition. ● More partitions in Kafka = More parallelism in Spark
  • 33. Achieving good throughput ● Provide the right # of partitions based on your cluster size for operations causing shuffles. words.map(x => (x, 1)).reduceByKey(_+_, 100) # of partitions
  • 34. Debugging a Streaming application Streaming tab in Spark UI
  • 35. Debugging a Streaming application Processing Time ● Make sure that the processing time < batch interval
  • 36. Debugging a Streaming application
  • 37. Debugging a Streaming application Batch Details Page: ● Input to the batch ● Jobs that were run as part of the processing for the batch
  • 38. Debugging a Streaming application Job Details Page ● DAG Visualization ● Stages of a Spark job
  • 39. Debugging a Streaming application Task Details Page Ensure that the tasks are executed on multiple executors (nodes) in your cluster to have enough parallelism while processing. If you have a single receiver, sometimes only one executor might be doing all the work though you have more than one executor in your cluster.
  • 40. Key benefits of Spark streaming
  • 42. Fast failure and Straggler recovery
  • 43. Combine Batch and Stream Processing Join data streams with static data sets val dataset = sparkContext.hadoopFile(“file”) … kafkaStream.transform{ batchRdd => batchRdd.join(dataset).filter(...) }
  • 44. Combine ML and Stream Processing Learn models offline, apply them online val model = KMeans.train(dataset, …) kakfaStream.map { event => model.predict(event.feature) }
  • 45. Combine SQL and Stream Processing inputStream.foreachRDD{ rdd => val df = SQLContext.createDataframe(rdd) df.select(...).where(...).groupBy(...) }