SlideShare a Scribd company logo
1 of 44
Download to read offline
Vida Ha & Holden Karau - Strata SJ 2015
Everyday I’m Shufflin
Tips for Writing Better Spark Jobs
Who are we?
Holden Karau
● Current: Software Engineer at Databricks.
● Co-author of “Learning Spark”.
● Past: Worked on search at Foursquare & Engineer @ Google.
Vida Ha
● Current: Solutions Engineer at Databricks.
● Past: Worked on scaling & distributed systems as a Software
Engineer at Square and Google.
Our assumptions
● You know what Apache Spark is.
● You have (or will) use it.
● You want to understand Spark’s internals, not
just it’s API’s, to write awesomer* Spark jobs.
*awesomer = more efficient, well-tested,
reusable, less error-prone, etc.
Key Takeaways
● Understanding the Shuffle in Spark
○ Common cause of inefficiency.
● Understanding when code runs on the driver
vs. the workers.
○ Common cause of errors.
● How to factor your code:
○ For reuse between batch & streaming.
○ For easy testing.
Good Old Word Count in Spark
sparkContext.textFile(“hdfs://…”)
.flatMap(lambda line: line.split())
.map(lambda word: (word, 1))
.reduceByKey(lambda a, b: a + b)
wordcount(wordcount) < 100
What about GroupbyKey instead?
sparkContext.textFile(“hdfs://…”)
.flatMap(lambda line: line.split())
.map(lambda word: (word, 1))
.groupByKey()
.map(lambda (w, counts): (w, sum(counts)))
Will we still get the right answer?
ReduceByKey vs. GroupByKey
Answer: Both will give you the same answer.
But reduceByKey is more efficient.
In fact, groupByKey can cause of out of disk
problems.
Examine the shuffleto understand.
sparkContext.textFile(“hdfs://…”)
.flatMap(lambda line: line.split())
.map(lambda word: (word, 1))
[reduceByKey or groupByKey]
...
What’s Happening in Word Count
Same
Worker
Node
Triggers
a shuffle
of data
Shuffle occurs to transfer all data with the same key
to the same worker node.
(a, 1)
(a, 2)
(a, 3)
(b, 1)
(b, 2)
(b, 3)
(a, 1)
(b, 1)
(a, 1)
(a, 1)
(b, 1)
(b, 1)
(a, 1)
(a, 1)
(a, 1)
(b, 1)
(b, 1)
(b, 1)
(a, 1)
(b, 1)
(a, 2)
(b, 2)
(a, 3)
(b, 3)
(a, 6) (b, 6)
ReduceByKey: Shuffle Step
Shuffle
With ReduceByKey, data is combined so each partition outputs at
most one value for each key to send over the network.
(a, 1)
(a, 1)
(a, 1)
(a, 1)
(a, 1)
(a, 1)
(b, 1)
(b, 1)
(b, 1)
(b, 1)
(b, 1)
(b, 1)
(a, 1)
(b, 1)
(a, 1)
(a, 1)
(b, 1)
(b, 1)
(a, 1)
(a, 1)
(a, 1)
(b, 1)
(b, 1)
(b, 1)
(a, 6) (b, 6)
Shuffle
With GroupByKey, all the data is wastefully sent over the network
and collected on the reduce workers.
GroupByKey: Shuffle Step
Prefer ReduceByKey over GroupByKey
Caveat: Not all problems that can be solved by
groupByKey can be calculated with reduceByKey.
ReduceByKey requires combining all your values
into another value with the exact same type.
reduceByKey, aggregateByKey, foldByKey, and
combineByKey, preferred over groupByKey
Join a Large Table with a Small Table
join_rdd = sqlContext.sql(“select *
FROM people_in_the_us
JOIN states
ON people_in_the_us.state = states.name”)
print join_rdd.toDebugString()
● ShuffledHashJoin?
● BroadcastHashJoin?
Even a larger Spark cluster will not solve these problems!
ShuffledHashJoin
US RDD
Partition 1
US RDD
Partition 2
US RDD
Partition n >> 50
Problems:
● Uneven
Sharding
● Limited
parallelism w/
50 output
partitions
Small
State
RDD
US RDD
Partition 2
US RDD
Partition 2
**All** the
Data for CA
**All** the
Data for RI
CA
RI
All the data
for the US will
be shuffled
into only 50
keys for each
of the states.
BroadcastHashJoin
Parallelism of the large RDD is maintained (n output
partitions), and shuffle is not even needed.
US RDD
Partition 1
US RDD
Partition 2
Small
State
RDD
US RDD
Partition n >> 50
Small
State
RDD
Small
State
RDD
Small
State
RDD
Solution: Broadcast the Small RDD to all worker nodes.
Broadcast
How to Configure BroadcastHashJoin
● See the Spark SQL programming guide for your
Spark version for how to configure.
● For Spark 1.2:
○ Set spark.sql.autoBroadcastJoinThreshold.
○ sqlContext.sql(“ANALYZE TABLE state_info
COMPUTE STATISTICS noscan”)
● Use .toDebugString() or EXPLAIN to double
check.
Join a Medium Table with a Huge Table
join_rdd = sqlContext.sql(“select *
FROM people_in_california
LEFT JOIN all_the_people_in_the_world
ON people_in_california.id =
all_the_people_in_the_world.id”)
Final output keys = keys people_in_california,
so this don’t need a huge Spark cluster, right?
Left Join - Shuffle Step
Not a Problem:
● Even Sharding
● Good Parallelism
Whole World
RDD
All Whole World
RDD
All CA
RDD
US RDD
Partition 2
All the Data from
Both Tables
Final
Joined
Output
The Size of the Spark Cluster to run this job is limited by the Large
table rather than the Medium Sized Table.
Shuffles everything
before dropping keys
What’s a Better Solution?
Whole World
RDD
Whole World
RDD
All CA
RDD
Final
Joined
Output
Filter the World World RDD for only entries that match the CA ID
Partial
World
RDD
Filter Transform
Benefits:
● Less Data shuffled
over the network
and less shuffle
space needed.
● More transforms,
but still faster.
Shuffle
What’s the Tipping Point for Huge?
● Can’t tell you.
● There aren’t always strict rules for optimizing.
● If you were only considering two small columns
from the World RDD in Parquet format, the
filtering step may not be worth it.
You should understand your data and it’s unique properties in
order to best optimize your Spark Job.
In Practice: Detecting Shuffle Problems
Things to Look for:
● Tasks that take much longer to run than others.
● Speculative tasks that are launching.
● Shards that have a lot more input or shuffle output than others.
Check the Spark UI
pages for task
level detail about
your Spark job.
Execution on the Driver vs. Workers
output = sparkContext
.textFile(“hdfs://…”)
.flatMap(lambda line: line.split())
.map(lambda word: (word, 1))
.reduceByKey(lambda a, b: a + b)
.collect()
print output …
Actions may transfer data from the Workers to the Driver.
The main program are executed on the Spark Driver.
Transformations are executed on the Spark Workers.
What happens when calling collect()
collect() on a large RDD can trigger a OOM error
collect() sends all the partitions to the single driver
Worker
Partition 1
Worker
Partition 2
Worker
Partition 3
Worker
Partition n
The Driver OOM Error!
Don’t call collect() on a large RDD
myLargeRdd.collect()
myLargeRdd.countByKey()
myLargeRdd.countByValue()
myLargeRdd.collectAsMap()
Be cautious with all
actions that may return
unbounded output.
Option 1: Choose actions
that return a bounded
output per partition, such
as count() or take(N) .
Option 2: Choose actions
that outputs directly from
the workers such as
saveAsTextFile().
Commonly Serialization Errors
Hadoop Writables
Capturing a full Non-
Serializable object
Map to/from a
serializable form
Copy the required
serializable parts locally
Network Connections Create the connection
on the worker
Serialization Error
myNonSerializable = …
output = sparkContext
.textFile(“hdfs://…”)
.map(lambda l: myNonSerializable.value + l)
.take(n)
print output …
Spark will try to send myNonSerializable from the Driver to the
Worker node by serializing it, and error.
RDDs within RDDs - not even once
Only the driver can perform operations on RDDs
map+get:
rdd.map{(key, value) => otherRdd.get(key)...}
can normally be replaced with a join:
rdd.join(otherRdd).map{}
map+map:
rdd.map{e => otherRdd.map{ … }}
is normally an attempt at a cartesian:
rdd.cartesian(otherRdd).map()
Writing a Large RDD to a Database
Option 1: DIY
● Initialize the Database
Connection on the Worker
rather than the Driver
○ Network sockets are
non-serializable
● Use foreachPartition
○ Re-use the connection
between elements
Option 2: DBOutputFormat
● Database must speak
JDBC
● Extend DBWritable and
save with
saveAsHadoopDataset
DIY: Large RDD to a Database
Cat photo from https://www.flickr.com/photos/rudiriet/140901529/
DIY: Large RDD to a Database
data.forEachPartition{records => {
// Create the connection on the executor
val connection = new HappyDatabase(...)
records.foreach{record =>
connection.//implementation specific
}
}
}
DBOutputFormat
case class CatRec(name: String, age: Int) extends DBWritable
{
override def write(s: PreparedStatement ) {
s.setString(1, name); s.setInt(2, age)
}}
val tableName = "table"
val fields = Array("name", "age")
val job = new JobConf()
DBConfiguration.configureDB(job, "com.mysql.jdbc.Driver",
"..")
DBOutputFormat.setOutput(job, tableName, fields:_*)
records.saveAsHadoopDataset(job)
Reuse Code on Batch & Streaming
val ips = logs.transform
(extractIp)
def extractIp(
logs: RDD[String]) = {
logs.map(_.split(“ “)(0))
}
val ips = extractIp(logs)
Streaming Batch
Use transform on a DStream to reuse your RDD to RDD functions
from your batch Spark jobs.
Reuse Code on Batch & Streaming
tweets.foreachRDD{(tweetRDD, time) =>
writeOutput(tweetRDD)
}
def writeOutput(ft..) = {
val preped = ft.map(prep)
preped.savetoEs(esResource)
}
val ft = tIds.mapPartition{
tweetP =>
val twttr = TwitterFactory.
getSingleton()
tweetP.map{ t =>
twttr.showStatus(t.toLong)
}}
writeOutput(ft)
Streaming Batch
Use foreachRDD on a DStream
to reuse your RDD output
functions from your batch
Spark jobs.
Testing Spark Programs
● Picture of a cat
● Unit-tests of functions
● Testing with RDDs
● Special Considerations for Streaming
Cat photo by Jason and Kris Carter
Simplest - Unit Test Functions
instead of:
val splitLines = inFile.map(line => {
val reader = new CSVReader(new StringReader(line))
reader.readNext()
})
write:
def parseLine(line: String): Array[Double] = {
val reader = new CSVReader(new StringReader(line))
reader.readNext().map(_.toDouble)
}
then we can:
test("should parse a csv line with numbers") {
MoreTestableLoadCsvExample.parseLine("1,2") should equal
(Array[Double](1.0, 2.0))
}
Testing with RDDs
trait SSC extends BeforeAndAfterAll { self: Suite =>
@transient private var _sc: SparkContext = _
def sc: SparkContext = _sc
var conf = new SparkConf(false)
override def beforeAll() {
_sc = new SparkContext("local[4]", "test", conf)
super.beforeAll()
}
override def afterAll() {
LocalSparkContext.stop(_sc)
_sc = null
super.afterAll()
}
}
Testing with RDDs
Or just include http://spark-packages.
org/package/holdenk/spark-testing-base
Link to spark-testing-base from spark-packages.
Testing with RDDs continued
test("should parse a csv line with numbers") {
val input = sc.parallelize(List("1,2"))
val result = input.map(parseCsvLine)
result.collect() should equal (Array[Double](1.0, 2.0))
}
Testing with DStreams
Some challenges:
● creating a test DStream
● collecting the data to compare against locally
○ use foreachRDD & a var
● stopping the streaming context after the input stream
is done
○ use a manual clock
■ (private class)
○ wait for a timeout
■ slow
Testing with DStreams - fun!
class SampleStreamingTest extends StreamingSuiteBase {
test("really simple transformation") {
val input = List(List("hi"), List("hi holden"),
List("bye"))
val expect = List(List("hi"),
List("hi”, “holden"),
List("bye"))
testOperation[String, String](input, tokenize _,
expect, useSet = true)
}
}
THE END
Cat picture from http://galato901.deviantart.com/art/Cat-on-Work-Break-173043455
The DAG - is it magic?
Dog photo from: Pets Adviser by http://petsadviser.com

More Related Content

What's hot

Apache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsApache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsAnton Kirillov
 
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin HuaiA Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin HuaiDatabricks
 
Fine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark JobsFine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark JobsDatabricks
 
Apache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationDatabricks
 
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark Summit
 
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudAmazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudNoritaka Sekiyama
 
Memory Management in Apache Spark
Memory Management in Apache SparkMemory Management in Apache Spark
Memory Management in Apache SparkDatabricks
 
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital KediaTuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital KediaDatabricks
 
Performance Troubleshooting Using Apache Spark Metrics
Performance Troubleshooting Using Apache Spark MetricsPerformance Troubleshooting Using Apache Spark Metrics
Performance Troubleshooting Using Apache Spark MetricsDatabricks
 
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...Databricks
 
What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?DataWorks Summit
 
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeSimplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeDatabricks
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDatabricks
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introductioncolorant
 
Deep Dive into the New Features of Apache Spark 3.0
Deep Dive into the New Features of Apache Spark 3.0Deep Dive into the New Features of Apache Spark 3.0
Deep Dive into the New Features of Apache Spark 3.0Databricks
 
Hudi: Large-Scale, Near Real-Time Pipelines at Uber with Nishith Agarwal and ...
Hudi: Large-Scale, Near Real-Time Pipelines at Uber with Nishith Agarwal and ...Hudi: Large-Scale, Near Real-Time Pipelines at Uber with Nishith Agarwal and ...
Hudi: Large-Scale, Near Real-Time Pipelines at Uber with Nishith Agarwal and ...Databricks
 
Programming in Spark using PySpark
Programming in Spark using PySpark      Programming in Spark using PySpark
Programming in Spark using PySpark Mostafa
 
Adaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at RuntimeAdaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at RuntimeDatabricks
 
Apache Spark on K8S Best Practice and Performance in the Cloud
Apache Spark on K8S Best Practice and Performance in the CloudApache Spark on K8S Best Practice and Performance in the Cloud
Apache Spark on K8S Best Practice and Performance in the CloudDatabricks
 

What's hot (20)

Apache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsApache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & Internals
 
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin HuaiA Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
A Deep Dive into Spark SQL's Catalyst Optimizer with Yin Huai
 
Fine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark JobsFine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark Jobs
 
Apache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical Optimization
 
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
 
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudAmazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
 
Memory Management in Apache Spark
Memory Management in Apache SparkMemory Management in Apache Spark
Memory Management in Apache Spark
 
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital KediaTuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
 
Performance Troubleshooting Using Apache Spark Metrics
Performance Troubleshooting Using Apache Spark MetricsPerformance Troubleshooting Using Apache Spark Metrics
Performance Troubleshooting Using Apache Spark Metrics
 
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
From Query Plan to Query Performance: Supercharging your Apache Spark Queries...
 
Apache Spark Architecture
Apache Spark ArchitectureApache Spark Architecture
Apache Spark Architecture
 
What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?
 
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeSimplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache Spark
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introduction
 
Deep Dive into the New Features of Apache Spark 3.0
Deep Dive into the New Features of Apache Spark 3.0Deep Dive into the New Features of Apache Spark 3.0
Deep Dive into the New Features of Apache Spark 3.0
 
Hudi: Large-Scale, Near Real-Time Pipelines at Uber with Nishith Agarwal and ...
Hudi: Large-Scale, Near Real-Time Pipelines at Uber with Nishith Agarwal and ...Hudi: Large-Scale, Near Real-Time Pipelines at Uber with Nishith Agarwal and ...
Hudi: Large-Scale, Near Real-Time Pipelines at Uber with Nishith Agarwal and ...
 
Programming in Spark using PySpark
Programming in Spark using PySpark      Programming in Spark using PySpark
Programming in Spark using PySpark
 
Adaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at RuntimeAdaptive Query Execution: Speeding Up Spark SQL at Runtime
Adaptive Query Execution: Speeding Up Spark SQL at Runtime
 
Apache Spark on K8S Best Practice and Performance in the Cloud
Apache Spark on K8S Best Practice and Performance in the CloudApache Spark on K8S Best Practice and Performance in the Cloud
Apache Spark on K8S Best Practice and Performance in the Cloud
 

Viewers also liked

Top 5 Mistakes to Avoid When Writing Apache Spark Applications
Top 5 Mistakes to Avoid When Writing Apache Spark ApplicationsTop 5 Mistakes to Avoid When Writing Apache Spark Applications
Top 5 Mistakes to Avoid When Writing Apache Spark ApplicationsCloudera, Inc.
 
Processing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeekProcessing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeekVenkata Naga Ravi
 
Sqoop on Spark for Data Ingestion
Sqoop on Spark for Data IngestionSqoop on Spark for Data Ingestion
Sqoop on Spark for Data IngestionDataWorks Summit
 
Advanced Data Science on Spark-(Reza Zadeh, Stanford)
Advanced Data Science on Spark-(Reza Zadeh, Stanford)Advanced Data Science on Spark-(Reza Zadeh, Stanford)
Advanced Data Science on Spark-(Reza Zadeh, Stanford)Spark Summit
 
Intro to Spark development
 Intro to Spark development  Intro to Spark development
Intro to Spark development Spark Summit
 
Optimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL JoinsOptimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL JoinsDatabricks
 
Spark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting GuideSpark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting GuideIBM
 

Viewers also liked (8)

Top 5 Mistakes to Avoid When Writing Apache Spark Applications
Top 5 Mistakes to Avoid When Writing Apache Spark ApplicationsTop 5 Mistakes to Avoid When Writing Apache Spark Applications
Top 5 Mistakes to Avoid When Writing Apache Spark Applications
 
Processing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeekProcessing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeek
 
Sqoop on Spark for Data Ingestion
Sqoop on Spark for Data IngestionSqoop on Spark for Data Ingestion
Sqoop on Spark for Data Ingestion
 
Advanced Data Science on Spark-(Reza Zadeh, Stanford)
Advanced Data Science on Spark-(Reza Zadeh, Stanford)Advanced Data Science on Spark-(Reza Zadeh, Stanford)
Advanced Data Science on Spark-(Reza Zadeh, Stanford)
 
Dev Ops Training
Dev Ops TrainingDev Ops Training
Dev Ops Training
 
Intro to Spark development
 Intro to Spark development  Intro to Spark development
Intro to Spark development
 
Optimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL JoinsOptimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL Joins
 
Spark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting GuideSpark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting Guide
 

Similar to Spark optimization tips for better jobs

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
 
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...Аліна Шепшелей
 
Intermachine Parallelism
Intermachine ParallelismIntermachine Parallelism
Intermachine ParallelismSri Prasanna
 
Extreme Apache Spark: how in 3 months we created a pipeline that can process ...
Extreme Apache Spark: how in 3 months we created a pipeline that can process ...Extreme Apache Spark: how in 3 months we created a pipeline that can process ...
Extreme Apache Spark: how in 3 months we created a pipeline that can process ...Josef A. Habdank
 
AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)Paul Chao
 
Berlin buzzwords 2018
Berlin buzzwords 2018Berlin buzzwords 2018
Berlin buzzwords 2018Matija Gobec
 
Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Massimo Schenone
 
Scala Meetup Hamburg - Spark
Scala Meetup Hamburg - SparkScala Meetup Hamburg - Spark
Scala Meetup Hamburg - SparkIvan Morozov
 
Hadoop Network Performance profile
Hadoop Network Performance profileHadoop Network Performance profile
Hadoop Network Performance profilepramodbiligiri
 
Productionizing your Streaming Jobs
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming JobsDatabricks
 
ACADILD:: HADOOP LESSON
ACADILD:: HADOOP LESSON ACADILD:: HADOOP LESSON
ACADILD:: HADOOP LESSON Padma shree. T
 
Apache Spark, the Next Generation Cluster Computing
Apache Spark, the Next Generation Cluster ComputingApache Spark, the Next Generation Cluster Computing
Apache Spark, the Next Generation Cluster ComputingGerger
 
Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...
Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...
Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...Holden Karau
 
No more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in productionNo more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in productionChetan Khatri
 
HDFS-HC: A Data Placement Module for Heterogeneous Hadoop Clusters
HDFS-HC: A Data Placement Module for Heterogeneous Hadoop ClustersHDFS-HC: A Data Placement Module for Heterogeneous Hadoop Clusters
HDFS-HC: A Data Placement Module for Heterogeneous Hadoop ClustersXiao Qin
 

Similar to Spark optimization tips for better jobs (20)

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
 
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...
 
Intermachine Parallelism
Intermachine ParallelismIntermachine Parallelism
Intermachine Parallelism
 
Extreme Apache Spark: how in 3 months we created a pipeline that can process ...
Extreme Apache Spark: how in 3 months we created a pipeline that can process ...Extreme Apache Spark: how in 3 months we created a pipeline that can process ...
Extreme Apache Spark: how in 3 months we created a pipeline that can process ...
 
AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)
 
Berlin buzzwords 2018
Berlin buzzwords 2018Berlin buzzwords 2018
Berlin buzzwords 2018
 
Apache Spark: What? Why? When?
Apache Spark: What? Why? When?Apache Spark: What? Why? When?
Apache Spark: What? Why? When?
 
Scala Meetup Hamburg - Spark
Scala Meetup Hamburg - SparkScala Meetup Hamburg - Spark
Scala Meetup Hamburg - Spark
 
Hadoop ecosystem
Hadoop ecosystemHadoop ecosystem
Hadoop ecosystem
 
Hadoop ecosystem
Hadoop ecosystemHadoop ecosystem
Hadoop ecosystem
 
Hadoop Network Performance profile
Hadoop Network Performance profileHadoop Network Performance profile
Hadoop Network Performance profile
 
Productionizing your Streaming Jobs
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming Jobs
 
ACADILD:: HADOOP LESSON
ACADILD:: HADOOP LESSON ACADILD:: HADOOP LESSON
ACADILD:: HADOOP LESSON
 
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 Spark, the Next Generation Cluster Computing
Apache Spark, the Next Generation Cluster ComputingApache Spark, the Next Generation Cluster Computing
Apache Spark, the Next Generation Cluster Computing
 
Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...
Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...
Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ...
 
No more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in productionNo more struggles with Apache Spark workloads in production
No more struggles with Apache Spark workloads in production
 
HDFS-HC: A Data Placement Module for Heterogeneous Hadoop Clusters
HDFS-HC: A Data Placement Module for Heterogeneous Hadoop ClustersHDFS-HC: A Data Placement Module for Heterogeneous Hadoop Clusters
HDFS-HC: A Data Placement Module for Heterogeneous Hadoop Clusters
 

More from 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
 

More from 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
 

Recently uploaded

Take control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteTake control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteDianaGray10
 
How AI, OpenAI, and ChatGPT impact business and software.
How AI, OpenAI, and ChatGPT impact business and software.How AI, OpenAI, and ChatGPT impact business and software.
How AI, OpenAI, and ChatGPT impact business and software.Curtis Poe
 
"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek Schlawack
"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek Schlawack"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek Schlawack
"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek SchlawackFwdays
 
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxDigital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxLoriGlavin3
 
"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii SoldatenkoFwdays
 
DSPy a system for AI to Write Prompts and Do Fine Tuning
DSPy a system for AI to Write Prompts and Do Fine TuningDSPy a system for AI to Write Prompts and Do Fine Tuning
DSPy a system for AI to Write Prompts and Do Fine TuningLars Bell
 
The Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and ConsThe Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and ConsPixlogix Infotech
 
SIP trunking in Janus @ Kamailio World 2024
SIP trunking in Janus @ Kamailio World 2024SIP trunking in Janus @ Kamailio World 2024
SIP trunking in Janus @ Kamailio World 2024Lorenzo Miniero
 
Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 365Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 3652toLead Limited
 
TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024Lonnie McRorey
 
What is Artificial Intelligence?????????
What is Artificial Intelligence?????????What is Artificial Intelligence?????????
What is Artificial Intelligence?????????blackmambaettijean
 
WordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your BrandWordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your Brandgvaughan
 
Dev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebDev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebUiPathCommunity
 
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024BookNet Canada
 
Advanced Computer Architecture – An Introduction
Advanced Computer Architecture – An IntroductionAdvanced Computer Architecture – An Introduction
Advanced Computer Architecture – An IntroductionDilum Bandara
 
Passkey Providers and Enabling Portability: FIDO Paris Seminar.pptx
Passkey Providers and Enabling Portability: FIDO Paris Seminar.pptxPasskey Providers and Enabling Portability: FIDO Paris Seminar.pptx
Passkey Providers and Enabling Portability: FIDO Paris Seminar.pptxLoriGlavin3
 
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024BookNet Canada
 
DevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenDevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenHervé Boutemy
 
Training state-of-the-art general text embedding
Training state-of-the-art general text embeddingTraining state-of-the-art general text embedding
Training state-of-the-art general text embeddingZilliz
 
Rise of the Machines: Known As Drones...
Rise of the Machines: Known As Drones...Rise of the Machines: Known As Drones...
Rise of the Machines: Known As Drones...Rick Flair
 

Recently uploaded (20)

Take control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteTake control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test Suite
 
How AI, OpenAI, and ChatGPT impact business and software.
How AI, OpenAI, and ChatGPT impact business and software.How AI, OpenAI, and ChatGPT impact business and software.
How AI, OpenAI, and ChatGPT impact business and software.
 
"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek Schlawack
"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek Schlawack"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek Schlawack
"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek Schlawack
 
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxDigital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
 
"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko
 
DSPy a system for AI to Write Prompts and Do Fine Tuning
DSPy a system for AI to Write Prompts and Do Fine TuningDSPy a system for AI to Write Prompts and Do Fine Tuning
DSPy a system for AI to Write Prompts and Do Fine Tuning
 
The Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and ConsThe Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and Cons
 
SIP trunking in Janus @ Kamailio World 2024
SIP trunking in Janus @ Kamailio World 2024SIP trunking in Janus @ Kamailio World 2024
SIP trunking in Janus @ Kamailio World 2024
 
Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 365Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 365
 
TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024
 
What is Artificial Intelligence?????????
What is Artificial Intelligence?????????What is Artificial Intelligence?????????
What is Artificial Intelligence?????????
 
WordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your BrandWordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your Brand
 
Dev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebDev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio Web
 
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
 
Advanced Computer Architecture – An Introduction
Advanced Computer Architecture – An IntroductionAdvanced Computer Architecture – An Introduction
Advanced Computer Architecture – An Introduction
 
Passkey Providers and Enabling Portability: FIDO Paris Seminar.pptx
Passkey Providers and Enabling Portability: FIDO Paris Seminar.pptxPasskey Providers and Enabling Portability: FIDO Paris Seminar.pptx
Passkey Providers and Enabling Portability: FIDO Paris Seminar.pptx
 
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
 
DevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenDevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache Maven
 
Training state-of-the-art general text embedding
Training state-of-the-art general text embeddingTraining state-of-the-art general text embedding
Training state-of-the-art general text embedding
 
Rise of the Machines: Known As Drones...
Rise of the Machines: Known As Drones...Rise of the Machines: Known As Drones...
Rise of the Machines: Known As Drones...
 

Spark optimization tips for better jobs

  • 1. Vida Ha & Holden Karau - Strata SJ 2015 Everyday I’m Shufflin Tips for Writing Better Spark Jobs
  • 2. Who are we? Holden Karau ● Current: Software Engineer at Databricks. ● Co-author of “Learning Spark”. ● Past: Worked on search at Foursquare & Engineer @ Google. Vida Ha ● Current: Solutions Engineer at Databricks. ● Past: Worked on scaling & distributed systems as a Software Engineer at Square and Google.
  • 3. Our assumptions ● You know what Apache Spark is. ● You have (or will) use it. ● You want to understand Spark’s internals, not just it’s API’s, to write awesomer* Spark jobs. *awesomer = more efficient, well-tested, reusable, less error-prone, etc.
  • 4. Key Takeaways ● Understanding the Shuffle in Spark ○ Common cause of inefficiency. ● Understanding when code runs on the driver vs. the workers. ○ Common cause of errors. ● How to factor your code: ○ For reuse between batch & streaming. ○ For easy testing.
  • 5. Good Old Word Count in Spark sparkContext.textFile(“hdfs://…”) .flatMap(lambda line: line.split()) .map(lambda word: (word, 1)) .reduceByKey(lambda a, b: a + b) wordcount(wordcount) < 100
  • 6. What about GroupbyKey instead? sparkContext.textFile(“hdfs://…”) .flatMap(lambda line: line.split()) .map(lambda word: (word, 1)) .groupByKey() .map(lambda (w, counts): (w, sum(counts))) Will we still get the right answer?
  • 7. ReduceByKey vs. GroupByKey Answer: Both will give you the same answer. But reduceByKey is more efficient. In fact, groupByKey can cause of out of disk problems. Examine the shuffleto understand.
  • 8. sparkContext.textFile(“hdfs://…”) .flatMap(lambda line: line.split()) .map(lambda word: (word, 1)) [reduceByKey or groupByKey] ... What’s Happening in Word Count Same Worker Node Triggers a shuffle of data Shuffle occurs to transfer all data with the same key to the same worker node.
  • 9. (a, 1) (a, 2) (a, 3) (b, 1) (b, 2) (b, 3) (a, 1) (b, 1) (a, 1) (a, 1) (b, 1) (b, 1) (a, 1) (a, 1) (a, 1) (b, 1) (b, 1) (b, 1) (a, 1) (b, 1) (a, 2) (b, 2) (a, 3) (b, 3) (a, 6) (b, 6) ReduceByKey: Shuffle Step Shuffle With ReduceByKey, data is combined so each partition outputs at most one value for each key to send over the network.
  • 10. (a, 1) (a, 1) (a, 1) (a, 1) (a, 1) (a, 1) (b, 1) (b, 1) (b, 1) (b, 1) (b, 1) (b, 1) (a, 1) (b, 1) (a, 1) (a, 1) (b, 1) (b, 1) (a, 1) (a, 1) (a, 1) (b, 1) (b, 1) (b, 1) (a, 6) (b, 6) Shuffle With GroupByKey, all the data is wastefully sent over the network and collected on the reduce workers. GroupByKey: Shuffle Step
  • 11. Prefer ReduceByKey over GroupByKey Caveat: Not all problems that can be solved by groupByKey can be calculated with reduceByKey. ReduceByKey requires combining all your values into another value with the exact same type. reduceByKey, aggregateByKey, foldByKey, and combineByKey, preferred over groupByKey
  • 12. Join a Large Table with a Small Table join_rdd = sqlContext.sql(“select * FROM people_in_the_us JOIN states ON people_in_the_us.state = states.name”) print join_rdd.toDebugString() ● ShuffledHashJoin? ● BroadcastHashJoin?
  • 13. Even a larger Spark cluster will not solve these problems! ShuffledHashJoin US RDD Partition 1 US RDD Partition 2 US RDD Partition n >> 50 Problems: ● Uneven Sharding ● Limited parallelism w/ 50 output partitions Small State RDD US RDD Partition 2 US RDD Partition 2 **All** the Data for CA **All** the Data for RI CA RI All the data for the US will be shuffled into only 50 keys for each of the states.
  • 14. BroadcastHashJoin Parallelism of the large RDD is maintained (n output partitions), and shuffle is not even needed. US RDD Partition 1 US RDD Partition 2 Small State RDD US RDD Partition n >> 50 Small State RDD Small State RDD Small State RDD Solution: Broadcast the Small RDD to all worker nodes. Broadcast
  • 15. How to Configure BroadcastHashJoin ● See the Spark SQL programming guide for your Spark version for how to configure. ● For Spark 1.2: ○ Set spark.sql.autoBroadcastJoinThreshold. ○ sqlContext.sql(“ANALYZE TABLE state_info COMPUTE STATISTICS noscan”) ● Use .toDebugString() or EXPLAIN to double check.
  • 16. Join a Medium Table with a Huge Table join_rdd = sqlContext.sql(“select * FROM people_in_california LEFT JOIN all_the_people_in_the_world ON people_in_california.id = all_the_people_in_the_world.id”) Final output keys = keys people_in_california, so this don’t need a huge Spark cluster, right?
  • 17. Left Join - Shuffle Step Not a Problem: ● Even Sharding ● Good Parallelism Whole World RDD All Whole World RDD All CA RDD US RDD Partition 2 All the Data from Both Tables Final Joined Output The Size of the Spark Cluster to run this job is limited by the Large table rather than the Medium Sized Table. Shuffles everything before dropping keys
  • 18. What’s a Better Solution? Whole World RDD Whole World RDD All CA RDD Final Joined Output Filter the World World RDD for only entries that match the CA ID Partial World RDD Filter Transform Benefits: ● Less Data shuffled over the network and less shuffle space needed. ● More transforms, but still faster. Shuffle
  • 19. What’s the Tipping Point for Huge? ● Can’t tell you. ● There aren’t always strict rules for optimizing. ● If you were only considering two small columns from the World RDD in Parquet format, the filtering step may not be worth it. You should understand your data and it’s unique properties in order to best optimize your Spark Job.
  • 20. In Practice: Detecting Shuffle Problems Things to Look for: ● Tasks that take much longer to run than others. ● Speculative tasks that are launching. ● Shards that have a lot more input or shuffle output than others. Check the Spark UI pages for task level detail about your Spark job.
  • 21. Execution on the Driver vs. Workers output = sparkContext .textFile(“hdfs://…”) .flatMap(lambda line: line.split()) .map(lambda word: (word, 1)) .reduceByKey(lambda a, b: a + b) .collect() print output … Actions may transfer data from the Workers to the Driver. The main program are executed on the Spark Driver. Transformations are executed on the Spark Workers.
  • 22. What happens when calling collect() collect() on a large RDD can trigger a OOM error collect() sends all the partitions to the single driver Worker Partition 1 Worker Partition 2 Worker Partition 3 Worker Partition n The Driver OOM Error!
  • 23. Don’t call collect() on a large RDD myLargeRdd.collect() myLargeRdd.countByKey() myLargeRdd.countByValue() myLargeRdd.collectAsMap() Be cautious with all actions that may return unbounded output. Option 1: Choose actions that return a bounded output per partition, such as count() or take(N) . Option 2: Choose actions that outputs directly from the workers such as saveAsTextFile().
  • 24. Commonly Serialization Errors Hadoop Writables Capturing a full Non- Serializable object Map to/from a serializable form Copy the required serializable parts locally Network Connections Create the connection on the worker
  • 25. Serialization Error myNonSerializable = … output = sparkContext .textFile(“hdfs://…”) .map(lambda l: myNonSerializable.value + l) .take(n) print output … Spark will try to send myNonSerializable from the Driver to the Worker node by serializing it, and error.
  • 26. RDDs within RDDs - not even once Only the driver can perform operations on RDDs map+get: rdd.map{(key, value) => otherRdd.get(key)...} can normally be replaced with a join: rdd.join(otherRdd).map{} map+map: rdd.map{e => otherRdd.map{ … }} is normally an attempt at a cartesian: rdd.cartesian(otherRdd).map()
  • 27. Writing a Large RDD to a Database Option 1: DIY ● Initialize the Database Connection on the Worker rather than the Driver ○ Network sockets are non-serializable ● Use foreachPartition ○ Re-use the connection between elements Option 2: DBOutputFormat ● Database must speak JDBC ● Extend DBWritable and save with saveAsHadoopDataset
  • 28. DIY: Large RDD to a Database Cat photo from https://www.flickr.com/photos/rudiriet/140901529/
  • 29. DIY: Large RDD to a Database data.forEachPartition{records => { // Create the connection on the executor val connection = new HappyDatabase(...) records.foreach{record => connection.//implementation specific } } }
  • 30. DBOutputFormat case class CatRec(name: String, age: Int) extends DBWritable { override def write(s: PreparedStatement ) { s.setString(1, name); s.setInt(2, age) }} val tableName = "table" val fields = Array("name", "age") val job = new JobConf() DBConfiguration.configureDB(job, "com.mysql.jdbc.Driver", "..") DBOutputFormat.setOutput(job, tableName, fields:_*) records.saveAsHadoopDataset(job)
  • 31. Reuse Code on Batch & Streaming val ips = logs.transform (extractIp) def extractIp( logs: RDD[String]) = { logs.map(_.split(“ “)(0)) } val ips = extractIp(logs) Streaming Batch Use transform on a DStream to reuse your RDD to RDD functions from your batch Spark jobs.
  • 32. Reuse Code on Batch & Streaming tweets.foreachRDD{(tweetRDD, time) => writeOutput(tweetRDD) } def writeOutput(ft..) = { val preped = ft.map(prep) preped.savetoEs(esResource) } val ft = tIds.mapPartition{ tweetP => val twttr = TwitterFactory. getSingleton() tweetP.map{ t => twttr.showStatus(t.toLong) }} writeOutput(ft) Streaming Batch Use foreachRDD on a DStream to reuse your RDD output functions from your batch Spark jobs.
  • 33. Testing Spark Programs ● Picture of a cat ● Unit-tests of functions ● Testing with RDDs ● Special Considerations for Streaming
  • 34. Cat photo by Jason and Kris Carter
  • 35. Simplest - Unit Test Functions instead of: val splitLines = inFile.map(line => { val reader = new CSVReader(new StringReader(line)) reader.readNext() }) write: def parseLine(line: String): Array[Double] = { val reader = new CSVReader(new StringReader(line)) reader.readNext().map(_.toDouble) } then we can: test("should parse a csv line with numbers") { MoreTestableLoadCsvExample.parseLine("1,2") should equal (Array[Double](1.0, 2.0)) }
  • 36. Testing with RDDs trait SSC extends BeforeAndAfterAll { self: Suite => @transient private var _sc: SparkContext = _ def sc: SparkContext = _sc var conf = new SparkConf(false) override def beforeAll() { _sc = new SparkContext("local[4]", "test", conf) super.beforeAll() } override def afterAll() { LocalSparkContext.stop(_sc) _sc = null super.afterAll() } }
  • 37. Testing with RDDs Or just include http://spark-packages. org/package/holdenk/spark-testing-base Link to spark-testing-base from spark-packages.
  • 38. Testing with RDDs continued test("should parse a csv line with numbers") { val input = sc.parallelize(List("1,2")) val result = input.map(parseCsvLine) result.collect() should equal (Array[Double](1.0, 2.0)) }
  • 39. Testing with DStreams Some challenges: ● creating a test DStream ● collecting the data to compare against locally ○ use foreachRDD & a var ● stopping the streaming context after the input stream is done ○ use a manual clock ■ (private class) ○ wait for a timeout ■ slow
  • 40. Testing with DStreams - fun! class SampleStreamingTest extends StreamingSuiteBase { test("really simple transformation") { val input = List(List("hi"), List("hi holden"), List("bye")) val expect = List(List("hi"), List("hi”, “holden"), List("bye")) testOperation[String, String](input, tokenize _, expect, useSet = true) } }
  • 42.
  • 43. Cat picture from http://galato901.deviantart.com/art/Cat-on-Work-Break-173043455
  • 44. The DAG - is it magic? Dog photo from: Pets Adviser by http://petsadviser.com