SlideShare uma empresa Scribd logo
1 de 44
Baixar para ler 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
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

Mais conteúdo relacionado

Mais procurados

Memory Management in Apache Spark
Memory Management in Apache SparkMemory Management in Apache Spark
Memory Management in Apache SparkDatabricks
 
Hudi architecture, fundamentals and capabilities
Hudi architecture, fundamentals and capabilitiesHudi architecture, fundamentals and capabilities
Hudi architecture, fundamentals and capabilitiesNishith Agarwal
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introductioncolorant
 
Apache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper OptimizationApache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper OptimizationDatabricks
 
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
 
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.
 
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...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...Databricks
 
Why your Spark Job is Failing
Why your Spark Job is FailingWhy your Spark Job is Failing
Why your Spark Job is FailingDataWorks Summit
 
Introducing DataFrames in Spark for Large Scale Data Science
Introducing DataFrames in Spark for Large Scale Data ScienceIntroducing DataFrames in Spark for Large Scale Data Science
Introducing DataFrames in Spark for Large Scale Data ScienceDatabricks
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiDatabricks
 
Apache Spark Introduction
Apache Spark IntroductionApache Spark Introduction
Apache Spark Introductionsudhakara st
 
Apache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationDatabricks
 
A Deep Dive into Query Execution Engine of Spark SQL
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 SQLDatabricks
 
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/AvroThe Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/AvroDatabricks
 
Hive Bucketing in Apache Spark with Tejas Patil
Hive Bucketing in Apache Spark with Tejas PatilHive Bucketing in Apache Spark with Tejas Patil
Hive Bucketing in Apache Spark with Tejas PatilDatabricks
 
Top 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsTop 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsSpark Summit
 
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Databricks
 
How We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IOHow We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IODatabricks
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Databricks
 
Introduction to PySpark
Introduction to PySparkIntroduction to PySpark
Introduction to PySparkRussell Jurney
 

Mais procurados (20)

Memory Management in Apache Spark
Memory Management in Apache SparkMemory Management in Apache Spark
Memory Management in Apache Spark
 
Hudi architecture, fundamentals and capabilities
Hudi architecture, fundamentals and capabilitiesHudi architecture, fundamentals and capabilities
Hudi architecture, fundamentals and capabilities
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introduction
 
Apache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper OptimizationApache Spark Core—Deep Dive—Proper Optimization
Apache Spark Core—Deep Dive—Proper Optimization
 
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
 
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
 
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...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
 
Why your Spark Job is Failing
Why your Spark Job is FailingWhy your Spark Job is Failing
Why your Spark Job is Failing
 
Introducing DataFrames in Spark for Large Scale Data Science
Introducing DataFrames in Spark for Large Scale Data ScienceIntroducing DataFrames in Spark for Large Scale Data Science
Introducing DataFrames in Spark for Large Scale Data Science
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and Hudi
 
Apache Spark Introduction
Apache Spark IntroductionApache Spark Introduction
Apache Spark Introduction
 
Apache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical Optimization
 
A Deep Dive into Query Execution Engine of Spark SQL
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
 
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/AvroThe Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
 
Hive Bucketing in Apache Spark with Tejas Patil
Hive Bucketing in Apache Spark with Tejas PatilHive Bucketing in Apache Spark with Tejas Patil
Hive Bucketing in Apache Spark with Tejas Patil
 
Top 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsTop 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark Applications
 
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
Deep Dive into Stateful Stream Processing in Structured Streaming with Tathag...
 
How We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IOHow We Optimize Spark SQL Jobs With parallel and sync IO
How We Optimize Spark SQL Jobs With parallel and sync IO
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
 
Introduction to PySpark
Introduction to PySparkIntroduction to PySpark
Introduction to PySpark
 

Destaque

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
 
Spark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting GuideSpark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting GuideIBM
 

Destaque (6)

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
 
Spark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting GuideSpark 2.x Troubleshooting Guide
Spark 2.x Troubleshooting Guide
 

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

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
 

Semelhante a Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San Jose 2015 (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
 

Mais de Databricks

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

Mais de Databricks (20)

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

Último

RAG Patterns and Vector Search in Generative AI
RAG Patterns and Vector Search in Generative AIRAG Patterns and Vector Search in Generative AI
RAG Patterns and Vector Search in Generative AIUdaiappa Ramachandran
 
COMPUTER 10: Lesson 7 - File Storage and Online Collaboration
COMPUTER 10: Lesson 7 - File Storage and Online CollaborationCOMPUTER 10: Lesson 7 - File Storage and Online Collaboration
COMPUTER 10: Lesson 7 - File Storage and Online Collaborationbruanjhuli
 
Crea il tuo assistente AI con lo Stregatto (open source python framework)
Crea il tuo assistente AI con lo Stregatto (open source python framework)Crea il tuo assistente AI con lo Stregatto (open source python framework)
Crea il tuo assistente AI con lo Stregatto (open source python framework)Commit University
 
IaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdf
IaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdfIaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdf
IaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdfDaniel Santiago Silva Capera
 
Do we need a new standard for visualizing the invisible?
Do we need a new standard for visualizing the invisible?Do we need a new standard for visualizing the invisible?
Do we need a new standard for visualizing the invisible?SANGHEE SHIN
 
Anypoint Code Builder , Google Pub sub connector and MuleSoft RPA
Anypoint Code Builder , Google Pub sub connector and MuleSoft RPAAnypoint Code Builder , Google Pub sub connector and MuleSoft RPA
Anypoint Code Builder , Google Pub sub connector and MuleSoft RPAshyamraj55
 
Basic Building Blocks of Internet of Things.
Basic Building Blocks of Internet of Things.Basic Building Blocks of Internet of Things.
Basic Building Blocks of Internet of Things.YounusS2
 
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyesHow to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyesThousandEyes
 
UiPath Platform: The Backend Engine Powering Your Automation - Session 1
UiPath Platform: The Backend Engine Powering Your Automation - Session 1UiPath Platform: The Backend Engine Powering Your Automation - Session 1
UiPath Platform: The Backend Engine Powering Your Automation - Session 1DianaGray10
 
20200723_insight_release_plan_v6.pdf20200723_insight_release_plan_v6.pdf
20200723_insight_release_plan_v6.pdf20200723_insight_release_plan_v6.pdf20200723_insight_release_plan_v6.pdf20200723_insight_release_plan_v6.pdf
20200723_insight_release_plan_v6.pdf20200723_insight_release_plan_v6.pdfJamie (Taka) Wang
 
PicPay - GenAI Finance Assistant - ChatGPT for Customer Service
PicPay - GenAI Finance Assistant - ChatGPT for Customer ServicePicPay - GenAI Finance Assistant - ChatGPT for Customer Service
PicPay - GenAI Finance Assistant - ChatGPT for Customer ServiceRenan Moreira de Oliveira
 
The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...
The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...
The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...Aggregage
 
Introduction to Quantum Computing
Introduction to Quantum ComputingIntroduction to Quantum Computing
Introduction to Quantum ComputingGDSC PJATK
 
Empowering Africa's Next Generation: The AI Leadership Blueprint
Empowering Africa's Next Generation: The AI Leadership BlueprintEmpowering Africa's Next Generation: The AI Leadership Blueprint
Empowering Africa's Next Generation: The AI Leadership BlueprintMahmoud Rabie
 
Babel Compiler - Transforming JavaScript for All Browsers.pptx
Babel Compiler - Transforming JavaScript for All Browsers.pptxBabel Compiler - Transforming JavaScript for All Browsers.pptx
Babel Compiler - Transforming JavaScript for All Browsers.pptxYounusS2
 
COMPUTER 10 Lesson 8 - Building a Website
COMPUTER 10 Lesson 8 - Building a WebsiteCOMPUTER 10 Lesson 8 - Building a Website
COMPUTER 10 Lesson 8 - Building a Websitedgelyza
 
UiPath Studio Web workshop series - Day 7
UiPath Studio Web workshop series - Day 7UiPath Studio Web workshop series - Day 7
UiPath Studio Web workshop series - Day 7DianaGray10
 
9 Steps For Building Winning Founding Team
9 Steps For Building Winning Founding Team9 Steps For Building Winning Founding Team
9 Steps For Building Winning Founding TeamAdam Moalla
 
Bird eye's view on Camunda open source ecosystem
Bird eye's view on Camunda open source ecosystemBird eye's view on Camunda open source ecosystem
Bird eye's view on Camunda open source ecosystemAsko Soukka
 
Cloud Revolution: Exploring the New Wave of Serverless Spatial Data
Cloud Revolution: Exploring the New Wave of Serverless Spatial DataCloud Revolution: Exploring the New Wave of Serverless Spatial Data
Cloud Revolution: Exploring the New Wave of Serverless Spatial DataSafe Software
 

Último (20)

RAG Patterns and Vector Search in Generative AI
RAG Patterns and Vector Search in Generative AIRAG Patterns and Vector Search in Generative AI
RAG Patterns and Vector Search in Generative AI
 
COMPUTER 10: Lesson 7 - File Storage and Online Collaboration
COMPUTER 10: Lesson 7 - File Storage and Online CollaborationCOMPUTER 10: Lesson 7 - File Storage and Online Collaboration
COMPUTER 10: Lesson 7 - File Storage and Online Collaboration
 
Crea il tuo assistente AI con lo Stregatto (open source python framework)
Crea il tuo assistente AI con lo Stregatto (open source python framework)Crea il tuo assistente AI con lo Stregatto (open source python framework)
Crea il tuo assistente AI con lo Stregatto (open source python framework)
 
IaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdf
IaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdfIaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdf
IaC & GitOps in a Nutshell - a FridayInANuthshell Episode.pdf
 
Do we need a new standard for visualizing the invisible?
Do we need a new standard for visualizing the invisible?Do we need a new standard for visualizing the invisible?
Do we need a new standard for visualizing the invisible?
 
Anypoint Code Builder , Google Pub sub connector and MuleSoft RPA
Anypoint Code Builder , Google Pub sub connector and MuleSoft RPAAnypoint Code Builder , Google Pub sub connector and MuleSoft RPA
Anypoint Code Builder , Google Pub sub connector and MuleSoft RPA
 
Basic Building Blocks of Internet of Things.
Basic Building Blocks of Internet of Things.Basic Building Blocks of Internet of Things.
Basic Building Blocks of Internet of Things.
 
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyesHow to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
 
UiPath Platform: The Backend Engine Powering Your Automation - Session 1
UiPath Platform: The Backend Engine Powering Your Automation - Session 1UiPath Platform: The Backend Engine Powering Your Automation - Session 1
UiPath Platform: The Backend Engine Powering Your Automation - Session 1
 
20200723_insight_release_plan_v6.pdf20200723_insight_release_plan_v6.pdf
20200723_insight_release_plan_v6.pdf20200723_insight_release_plan_v6.pdf20200723_insight_release_plan_v6.pdf20200723_insight_release_plan_v6.pdf
20200723_insight_release_plan_v6.pdf20200723_insight_release_plan_v6.pdf
 
PicPay - GenAI Finance Assistant - ChatGPT for Customer Service
PicPay - GenAI Finance Assistant - ChatGPT for Customer ServicePicPay - GenAI Finance Assistant - ChatGPT for Customer Service
PicPay - GenAI Finance Assistant - ChatGPT for Customer Service
 
The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...
The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...
The Data Metaverse: Unpacking the Roles, Use Cases, and Tech Trends in Data a...
 
Introduction to Quantum Computing
Introduction to Quantum ComputingIntroduction to Quantum Computing
Introduction to Quantum Computing
 
Empowering Africa's Next Generation: The AI Leadership Blueprint
Empowering Africa's Next Generation: The AI Leadership BlueprintEmpowering Africa's Next Generation: The AI Leadership Blueprint
Empowering Africa's Next Generation: The AI Leadership Blueprint
 
Babel Compiler - Transforming JavaScript for All Browsers.pptx
Babel Compiler - Transforming JavaScript for All Browsers.pptxBabel Compiler - Transforming JavaScript for All Browsers.pptx
Babel Compiler - Transforming JavaScript for All Browsers.pptx
 
COMPUTER 10 Lesson 8 - Building a Website
COMPUTER 10 Lesson 8 - Building a WebsiteCOMPUTER 10 Lesson 8 - Building a Website
COMPUTER 10 Lesson 8 - Building a Website
 
UiPath Studio Web workshop series - Day 7
UiPath Studio Web workshop series - Day 7UiPath Studio Web workshop series - Day 7
UiPath Studio Web workshop series - Day 7
 
9 Steps For Building Winning Founding Team
9 Steps For Building Winning Founding Team9 Steps For Building Winning Founding Team
9 Steps For Building Winning Founding Team
 
Bird eye's view on Camunda open source ecosystem
Bird eye's view on Camunda open source ecosystemBird eye's view on Camunda open source ecosystem
Bird eye's view on Camunda open source ecosystem
 
Cloud Revolution: Exploring the New Wave of Serverless Spatial Data
Cloud Revolution: Exploring the New Wave of Serverless Spatial DataCloud Revolution: Exploring the New Wave of Serverless Spatial Data
Cloud Revolution: Exploring the New Wave of Serverless Spatial Data
 

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