Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San Jose 2015

Databricks
DatabricksDeveloper Marketing and Relations at MuleSoft
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
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San Jose 2015
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
1 of 44

Recommended

Top 5 mistakes when writing Spark applications by
Top 5 mistakes when writing Spark applicationsTop 5 mistakes when writing Spark applications
Top 5 mistakes when writing Spark applicationshadooparchbook
14.6K views75 slides
Spark shuffle introduction by
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introductioncolorant
50.6K views33 slides
Memory Management in Apache Spark by
Memory Management in Apache SparkMemory Management in Apache Spark
Memory Management in Apache SparkDatabricks
9.7K views59 slides
Deep Dive: Memory Management in Apache Spark by
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDatabricks
14.5K views54 slides
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud by
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
33.3K views60 slides
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark by
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in SparkSpark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in SparkBo Yang
2.7K views19 slides

More Related Content

What's hot

Apache Spark Core—Deep Dive—Proper Optimization by
Apache Spark Core—Deep Dive—Proper OptimizationApache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper OptimizationDatabricks
6.1K views50 slides
Understanding Query Plans and Spark UIs by
Understanding Query Plans and Spark UIsUnderstanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsDatabricks
4.7K views50 slides
Spark Performance Tuning .pdf by
Spark Performance Tuning .pdfSpark Performance Tuning .pdf
Spark Performance Tuning .pdfAmit Raj
296 views20 slides
Optimizing Apache Spark SQL Joins by
Optimizing Apache Spark SQL JoinsOptimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL JoinsDatabricks
44.9K views24 slides
Apache Spark Core – Practical Optimization by
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationDatabricks
2.5K views40 slides
How to Actually Tune Your Spark Jobs So They Work by
How to Actually Tune Your Spark Jobs So They WorkHow to Actually Tune Your Spark Jobs So They Work
How to Actually Tune Your Spark Jobs So They WorkIlya Ganelin
8.6K views20 slides

What's hot(20)

Apache Spark Core—Deep Dive—Proper Optimization by Databricks
Apache Spark Core—Deep Dive—Proper OptimizationApache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper Optimization
Databricks6.1K views
Understanding Query Plans and Spark UIs by Databricks
Understanding Query Plans and Spark UIsUnderstanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIs
Databricks4.7K views
Spark Performance Tuning .pdf by Amit Raj
Spark Performance Tuning .pdfSpark Performance Tuning .pdf
Spark Performance Tuning .pdf
Amit Raj296 views
Optimizing Apache Spark SQL Joins by Databricks
Optimizing Apache Spark SQL JoinsOptimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL Joins
Databricks44.9K views
Apache Spark Core – Practical Optimization by Databricks
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical Optimization
Databricks2.5K views
How to Actually Tune Your Spark Jobs So They Work by Ilya Ganelin
How to Actually Tune Your Spark Jobs So They WorkHow to Actually Tune Your Spark Jobs So They Work
How to Actually Tune Your Spark Jobs So They Work
Ilya Ganelin8.6K views
Parallelizing with Apache Spark in Unexpected Ways by Databricks
Parallelizing with Apache Spark in Unexpected WaysParallelizing with Apache Spark in Unexpected Ways
Parallelizing with Apache Spark in Unexpected Ways
Databricks5.6K views
Flash for Apache Spark Shuffle with Cosco by Databricks
Flash for Apache Spark Shuffle with CoscoFlash for Apache Spark Shuffle with Cosco
Flash for Apache Spark Shuffle with Cosco
Databricks709 views
Apache Spark in Depth: Core Concepts, Architecture & Internals by Anton Kirillov
Apache Spark in Depth: Core Concepts, Architecture & InternalsApache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & Internals
Anton Kirillov9.6K views
Designing Structured Streaming Pipelines—How to Architect Things Right by Databricks
Designing Structured Streaming Pipelines—How to Architect Things RightDesigning Structured Streaming Pipelines—How to Architect Things Right
Designing Structured Streaming Pipelines—How to Architect Things Right
Databricks4.5K views
Hive Bucketing in Apache Spark with Tejas Patil by Databricks
Hive Bucketing in Apache Spark with Tejas PatilHive Bucketing in Apache Spark with Tejas Patil
Hive Bucketing in Apache Spark with Tejas Patil
Databricks14.9K views
A Deep Dive into Query Execution Engine of Spark SQL by Databricks
A Deep Dive into Query Execution Engine of Spark SQLA Deep Dive into Query Execution Engine of Spark SQL
A Deep Dive into Query Execution Engine of Spark SQL
Databricks6.6K views
Physical Plans in Spark SQL by Databricks
Physical Plans in Spark SQLPhysical Plans in Spark SQL
Physical Plans in Spark SQL
Databricks7K views
Tuning and Debugging in Apache Spark by Patrick Wendell
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
Patrick Wendell36K views
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P... by Databricks
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Databricks1K views
Fine Tuning and Enhancing Performance of Apache Spark Jobs by Databricks
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
Databricks2.5K views
Top 5 Mistakes When Writing Spark Applications by Spark Summit
Top 5 Mistakes When Writing Spark ApplicationsTop 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark Applications
Spark Summit26.4K views
Best Practices for Enabling Speculative Execution on Large Scale Platforms by Databricks
Best Practices for Enabling Speculative Execution on Large Scale PlatformsBest Practices for Enabling Speculative Execution on Large Scale Platforms
Best Practices for Enabling Speculative Execution on Large Scale Platforms
Databricks545 views
Radical Speed for SQL Queries on Databricks: Photon Under the Hood by Databricks
Radical Speed for SQL Queries on Databricks: Photon Under the HoodRadical Speed for SQL Queries on Databricks: Photon Under the Hood
Radical Speed for SQL Queries on Databricks: Photon Under the Hood
Databricks1.1K views

Viewers also liked

Top 5 Mistakes to Avoid When Writing Apache Spark Applications by
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.
127.8K views69 slides
Processing Large Data with Apache Spark -- HasGeek by
Processing Large Data with Apache Spark -- HasGeekProcessing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeekVenkata Naga Ravi
12.4K views72 slides
Sqoop on Spark for Data Ingestion by
Sqoop on Spark for Data IngestionSqoop on Spark for Data Ingestion
Sqoop on Spark for Data IngestionDataWorks Summit
16.3K views42 slides
Advanced Data Science on Spark-(Reza Zadeh, Stanford) by
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
4.8K views47 slides
Dev Ops Training by
Dev Ops TrainingDev Ops Training
Dev Ops TrainingSpark Summit
8.6K views301 slides
Intro to Spark development by
 Intro to Spark development  Intro to Spark development
Intro to Spark development Spark Summit
10K views175 slides

Viewers also liked(7)

Top 5 Mistakes to Avoid When Writing Apache Spark Applications by Cloudera, Inc.
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
Cloudera, Inc.127.8K views
Processing Large Data with Apache Spark -- HasGeek by Venkata Naga Ravi
Processing Large Data with Apache Spark -- HasGeekProcessing Large Data with Apache Spark -- HasGeek
Processing Large Data with Apache Spark -- HasGeek
Venkata Naga Ravi12.4K views
Sqoop on Spark for Data Ingestion by DataWorks Summit
Sqoop on Spark for Data IngestionSqoop on Spark for Data Ingestion
Sqoop on Spark for Data Ingestion
DataWorks Summit16.3K views
Advanced Data Science on Spark-(Reza Zadeh, Stanford) by Spark 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)
Spark Summit4.8K views
Intro to Spark development by Spark Summit
 Intro to Spark development  Intro to Spark development
Intro to Spark development
Spark Summit10K views
Spark 2.x Troubleshooting Guide by IBM
Spark 2.x Troubleshooting GuideSpark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting Guide
IBM51.1K views

Similar to Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San Jose 2015

Tuning and Debugging in Apache Spark by
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkDatabricks
12.3K views50 slides
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D... by
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
994 views39 slides
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi... by
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...Аліна Шепшелей
427 views39 slides
Intermachine Parallelism by
Intermachine ParallelismIntermachine Parallelism
Intermachine ParallelismSri Prasanna
666 views30 slides
Extreme Apache Spark: how in 3 months we created a pipeline that can process ... by
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
25.4K views65 slides
AI與大數據數據處理 Spark實戰(20171216) by
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)Paul Chao
1.1K views127 slides

Similar to Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San Jose 2015(20)

Tuning and Debugging in Apache Spark by Databricks
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
Databricks12.3K views
SE2016 BigData Vitalii Bondarenko "HD insight spark. Advanced in-memory Big D... by Inhacking
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...
Inhacking994 views
Vitalii Bondarenko HDinsight: spark. advanced in memory big-data analytics wi... by Аліна Шепшелей
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 by Sri Prasanna
Intermachine ParallelismIntermachine Parallelism
Intermachine Parallelism
Sri Prasanna666 views
Extreme Apache Spark: how in 3 months we created a pipeline that can process ... by Josef A. Habdank
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. Habdank25.4K views
AI與大數據數據處理 Spark實戰(20171216) by Paul Chao
AI與大數據數據處理 Spark實戰(20171216)AI與大數據數據處理 Spark實戰(20171216)
AI與大數據數據處理 Spark實戰(20171216)
Paul Chao1.1K views
Scala Meetup Hamburg - Spark by Ivan Morozov
Scala Meetup Hamburg - SparkScala Meetup Hamburg - Spark
Scala Meetup Hamburg - Spark
Ivan Morozov825 views
Hadoop Network Performance profile by pramodbiligiri
Hadoop Network Performance profileHadoop Network Performance profile
Hadoop Network Performance profile
pramodbiligiri1.1K views
Productionizing your Streaming Jobs by Databricks
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming Jobs
Databricks1.6K views
Apache Spark, the Next Generation Cluster Computing by Gerger
Apache Spark, the Next Generation Cluster ComputingApache Spark, the Next Generation Cluster Computing
Apache Spark, the Next Generation Cluster Computing
Gerger716 views
Beyond Shuffling, Tips and Tricks for Scaling Apache Spark updated for Spark ... by Holden Karau
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 Karau1.4K views
No more struggles with Apache Spark workloads in production by Chetan Khatri
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
Chetan Khatri168 views
HDFS-HC: A Data Placement Module for Heterogeneous Hadoop Clusters by Xiao Qin
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
Xiao Qin2.4K views
クラウドDWHとしても進化を続けるPivotal Greenplumご紹介 by Masayuki Matsushita
クラウドDWHとしても進化を続けるPivotal Greenplumご紹介クラウドDWHとしても進化を続けるPivotal Greenplumご紹介
クラウドDWHとしても進化を続けるPivotal Greenplumご紹介

More from Databricks

DW Migration Webinar-March 2022.pptx by
DW Migration Webinar-March 2022.pptxDW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptxDatabricks
4.3K views25 slides
Data Lakehouse Symposium | Day 1 | Part 1 by
Data Lakehouse Symposium | Day 1 | Part 1Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1Databricks
1.5K views43 slides
Data Lakehouse Symposium | Day 1 | Part 2 by
Data Lakehouse Symposium | Day 1 | Part 2Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2Databricks
741 views16 slides
Data Lakehouse Symposium | Day 4 by
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4Databricks
1.8K views74 slides
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop by
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
6.3K views64 slides
Democratizing Data Quality Through a Centralized Platform by
Democratizing Data Quality Through a Centralized PlatformDemocratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized PlatformDatabricks
1.4K views36 slides

More from Databricks(20)

DW Migration Webinar-March 2022.pptx by Databricks
DW Migration Webinar-March 2022.pptxDW Migration Webinar-March 2022.pptx
DW Migration Webinar-March 2022.pptx
Databricks4.3K views
Data Lakehouse Symposium | Day 1 | Part 1 by Databricks
Data Lakehouse Symposium | Day 1 | Part 1Data Lakehouse Symposium | Day 1 | Part 1
Data Lakehouse Symposium | Day 1 | Part 1
Databricks1.5K views
Data Lakehouse Symposium | Day 1 | Part 2 by Databricks
Data Lakehouse Symposium | Day 1 | Part 2Data Lakehouse Symposium | Day 1 | Part 2
Data Lakehouse Symposium | Day 1 | Part 2
Databricks741 views
Data Lakehouse Symposium | Day 4 by Databricks
Data Lakehouse Symposium | Day 4Data Lakehouse Symposium | Day 4
Data Lakehouse Symposium | Day 4
Databricks1.8K views
5 Critical Steps to Clean Your Data Swamp When Migrating Off of Hadoop by Databricks
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
Databricks6.3K views
Democratizing Data Quality Through a Centralized Platform by Databricks
Democratizing Data Quality Through a Centralized PlatformDemocratizing Data Quality Through a Centralized Platform
Democratizing Data Quality Through a Centralized Platform
Databricks1.4K views
Learn to Use Databricks for Data Science by Databricks
Learn to Use Databricks for Data ScienceLearn to Use Databricks for Data Science
Learn to Use Databricks for Data Science
Databricks1.6K views
Why APM Is Not the Same As ML Monitoring by Databricks
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
Databricks743 views
The Function, the Context, and the Data—Enabling ML Ops at Stitch Fix by Databricks
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
Databricks689 views
Stage Level Scheduling Improving Big Data and AI Integration by Databricks
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
Databricks850 views
Simplify Data Conversion from Spark to TensorFlow and PyTorch by Databricks
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
Databricks1.8K views
Scaling your Data Pipelines with Apache Spark on Kubernetes by Databricks
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
Databricks2.1K views
Scaling and Unifying SciKit Learn and Apache Spark Pipelines by Databricks
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
Databricks667 views
Sawtooth Windows for Feature Aggregations by Databricks
Sawtooth Windows for Feature AggregationsSawtooth Windows for Feature Aggregations
Sawtooth Windows for Feature Aggregations
Databricks605 views
Redis + Apache Spark = Swiss Army Knife Meets Kitchen Sink by Databricks
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
Databricks677 views
Re-imagine Data Monitoring with whylogs and Spark by Databricks
Re-imagine Data Monitoring with whylogs and SparkRe-imagine Data Monitoring with whylogs and Spark
Re-imagine Data Monitoring with whylogs and Spark
Databricks550 views
Raven: End-to-end Optimization of ML Prediction Queries by Databricks
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
Databricks449 views
Processing Large Datasets for ADAS Applications using Apache Spark by Databricks
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
Databricks513 views
Massive Data Processing in Adobe Using Delta Lake by Databricks
Massive Data Processing in Adobe Using Delta LakeMassive Data Processing in Adobe Using Delta Lake
Massive Data Processing in Adobe Using Delta Lake
Databricks719 views
Machine Learning CI/CD for Email Attack Detection by Databricks
Machine Learning CI/CD for Email Attack DetectionMachine Learning CI/CD for Email Attack Detection
Machine Learning CI/CD for Email Attack Detection
Databricks389 views

Recently uploaded

NET Conf 2023 Recap by
NET Conf 2023 RecapNET Conf 2023 Recap
NET Conf 2023 RecapLee Richardson
10 views71 slides
Scaling Knowledge Graph Architectures with AI by
Scaling Knowledge Graph Architectures with AIScaling Knowledge Graph Architectures with AI
Scaling Knowledge Graph Architectures with AIEnterprise Knowledge
38 views15 slides
MVP and prioritization.pdf by
MVP and prioritization.pdfMVP and prioritization.pdf
MVP and prioritization.pdfrahuldharwal141
31 views8 slides
Ransomware is Knocking your Door_Final.pdf by
Ransomware is Knocking your Door_Final.pdfRansomware is Knocking your Door_Final.pdf
Ransomware is Knocking your Door_Final.pdfSecurity Bootcamp
59 views46 slides
Microsoft Power Platform.pptx by
Microsoft Power Platform.pptxMicrosoft Power Platform.pptx
Microsoft Power Platform.pptxUni Systems S.M.S.A.
53 views38 slides
Piloting & Scaling Successfully With Microsoft Viva by
Piloting & Scaling Successfully With Microsoft VivaPiloting & Scaling Successfully With Microsoft Viva
Piloting & Scaling Successfully With Microsoft VivaRichard Harbridge
12 views160 slides

Recently uploaded(20)

Piloting & Scaling Successfully With Microsoft Viva by Richard Harbridge
Piloting & Scaling Successfully With Microsoft VivaPiloting & Scaling Successfully With Microsoft Viva
Piloting & Scaling Successfully With Microsoft Viva
Special_edition_innovator_2023.pdf by WillDavies22
Special_edition_innovator_2023.pdfSpecial_edition_innovator_2023.pdf
Special_edition_innovator_2023.pdf
WillDavies2218 views
SAP Automation Using Bar Code and FIORI.pdf by Virendra Rai, PMP
SAP Automation Using Bar Code and FIORI.pdfSAP Automation Using Bar Code and FIORI.pdf
SAP Automation Using Bar Code and FIORI.pdf
"Running students' code in isolation. The hard way", Yurii Holiuk by Fwdays
"Running students' code in isolation. The hard way", Yurii Holiuk "Running students' code in isolation. The hard way", Yurii Holiuk
"Running students' code in isolation. The hard way", Yurii Holiuk
Fwdays17 views
STKI Israeli Market Study 2023 corrected forecast 2023_24 v3.pdf by Dr. Jimmy Schwarzkopf
STKI Israeli Market Study 2023   corrected forecast 2023_24 v3.pdfSTKI Israeli Market Study 2023   corrected forecast 2023_24 v3.pdf
STKI Israeli Market Study 2023 corrected forecast 2023_24 v3.pdf
【USB韌體設計課程】精選講義節錄-USB的列舉過程_艾鍗學院 by IttrainingIttraining
【USB韌體設計課程】精選講義節錄-USB的列舉過程_艾鍗學院【USB韌體設計課程】精選講義節錄-USB的列舉過程_艾鍗學院
【USB韌體設計課程】精選講義節錄-USB的列舉過程_艾鍗學院
Powerful Google developer tools for immediate impact! (2023-24) by wesley chun
Powerful Google developer tools for immediate impact! (2023-24)Powerful Google developer tools for immediate impact! (2023-24)
Powerful Google developer tools for immediate impact! (2023-24)
wesley chun10 views
ESPC 2023 - Protect and Govern your Sensitive Data with Microsoft Purview in ... by Jasper Oosterveld
ESPC 2023 - Protect and Govern your Sensitive Data with Microsoft Purview in ...ESPC 2023 - Protect and Govern your Sensitive Data with Microsoft Purview in ...
ESPC 2023 - Protect and Govern your Sensitive Data with Microsoft Purview in ...
STPI OctaNE CoE Brochure.pdf by madhurjyapb
STPI OctaNE CoE Brochure.pdfSTPI OctaNE CoE Brochure.pdf
STPI OctaNE CoE Brochure.pdf
madhurjyapb14 views

Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San Jose 2015

  • 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) } }
  • 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